Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[WIP] Update xlang kinesis to v2 #33416

Draft
wants to merge 22 commits into
base: master
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/trigger_files/beam_PostCommit_Python.json
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 7
"modification": 8
}

Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,9 @@
import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
Expand Down Expand Up @@ -56,7 +59,13 @@ public static byte[] serializeToByteArray(Serializable value) {
}
return buffer.toByteArray();
} catch (IOException exn) {
throw new IllegalArgumentException("unable to serialize " + value, exn);
ObjectWriter ow = new ObjectMapper().writer().withDefaultPrettyPrinter();
try {
throw new IllegalArgumentException(
"unable to serialize " + ow.writeValueAsString(value), exn);
} catch (JsonProcessingException ex) {
throw new IllegalArgumentException("unable to jsonify " + value, exn);
}
}
}

Expand Down
39 changes: 39 additions & 0 deletions sdks/java/io/amazon-web-services2/expansion-service/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* License); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an AS IS BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

apply plugin: 'org.apache.beam.module'
apply plugin: 'application'
mainClassName = "org.apache.beam.sdk.expansion.service.ExpansionService"

applyJavaNature(
automaticModuleName: 'org.apache.beam.sdk.io.amazon-web-services2.expansion.service',
exportJavadoc: false,
validateShadowJar: false,
shadowClosure: {},
)

description = "Apache Beam :: SDKs :: Java :: IO :: Amazon Web Services 2 :: Expansion Service"
ext.summary = "Expansion service serving AWS2"

dependencies {
implementation project(":sdks:java:expansion-service")
permitUnusedDeclared project(":sdks:java:expansion-service")
implementation project(":sdks:java:io:amazon-web-services2")
permitUnusedDeclared project(":sdks:java:io:amazon-web-services2")
runtimeOnly library.java.slf4j_jdk14
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,312 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.sdk.io.aws2.kinesis;

import com.google.auto.service.AutoService;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
import org.apache.beam.sdk.io.aws2.common.ClientConfiguration;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ExternalTransformBuilder;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
import org.joda.time.Instant;
import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
import software.amazon.awssdk.regions.Region;
import software.amazon.kinesis.common.InitialPositionInStream;

/**
* Exposes {@link org.apache.beam.sdk.io.aws2.kinesis.KinesisIO.Write} and {@link
* org.apache.beam.sdk.io.aws2.kinesis.KinesisIO.Read} as an external transform for cross-language
* usage.
*/
@AutoService(ExternalTransformRegistrar.class)
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
public class KinesisTransformRegistrar implements ExternalTransformRegistrar {
public static final String WRITE_URN = "beam:transform:org.apache.beam:kinesis_write:v2";
public static final String READ_DATA_URN = "beam:transform:org.apache.beam:kinesis_read_data:v2";

@Override
public Map<String, ExternalTransformBuilder<?, ?, ?>> knownBuilderInstances() {
return ImmutableMap.of(WRITE_URN, new WriteBuilder(), READ_DATA_URN, new ReadDataBuilder());
}

private abstract static class CrossLanguageConfiguration {
String streamName;
String awsAccessKey;
String awsSecretKey;
String region;
@Nullable String serviceEndpoint;

public void setStreamName(String streamName) {
this.streamName = streamName;
}

public void setAwsAccessKey(String awsAccessKey) {
this.awsAccessKey = awsAccessKey;
}

public void setAwsSecretKey(String awsSecretKey) {
this.awsSecretKey = awsSecretKey;
}

public void setRegion(String region) {
this.region = region;
}

public void setServiceEndpoint(@Nullable String serviceEndpoint) {
this.serviceEndpoint = serviceEndpoint;
}
}

public static class WriteBuilder
implements ExternalTransformBuilder<
WriteBuilder.Configuration, PCollection<byte[]>, KinesisIO.Write.Result> {

public static class Configuration extends CrossLanguageConfiguration {
private String partitionKey;

public void setPartitionKey(String partitionKey) {
this.partitionKey = partitionKey;
}
}

@Override
public PTransform<PCollection<byte[]>, KinesisIO.Write.Result> buildExternal(
Configuration configuration) {
AwsBasicCredentials creds =
AwsBasicCredentials.create(configuration.awsAccessKey, configuration.awsSecretKey);
StaticCredentialsProvider provider = StaticCredentialsProvider.create(creds);
SerializableFunction<byte[], byte[]> serializer = v -> v;
@Nullable URI endpoint = null;
if (configuration.serviceEndpoint != null) {
try {
endpoint = new URI(configuration.serviceEndpoint);
} catch (URISyntaxException ex) {
throw new RuntimeException(
String.format(
"Service endpoint must be URI format, got: %s", configuration.serviceEndpoint));
}
}
KinesisIO.Write<byte[]> writeTransform =
KinesisIO.<byte[]>write()
.withStreamName(configuration.streamName)
.withClientConfiguration(
ClientConfiguration.builder()
.credentialsProvider(provider)
.region(Region.of(configuration.region))
.endpoint(endpoint)
.build())
.withPartitioner(p -> configuration.partitionKey)
.withSerializer(serializer);

return writeTransform;
}
}

public static class ReadDataBuilder
implements ExternalTransformBuilder<
ReadDataBuilder.Configuration, PBegin, PCollection<byte[]>> {

public static class Configuration extends CrossLanguageConfiguration {
private @Nullable Long maxNumRecords;
private @Nullable Duration maxReadTime;
private @Nullable InitialPositionInStream initialPositionInStream;
private @Nullable Instant initialTimestampInStream;
private @Nullable Integer requestRecordsLimit;
private @Nullable Duration upToDateThreshold;
private @Nullable Long maxCapacityPerShard;
private @Nullable WatermarkPolicy watermarkPolicy;
private @Nullable Duration watermarkIdleDurationThreshold;
private @Nullable Duration rateLimit;

public void setMaxNumRecords(@Nullable Long maxNumRecords) {
this.maxNumRecords = maxNumRecords;
}

public void setMaxReadTime(@Nullable Long maxReadTime) {
if (maxReadTime != null) {
this.maxReadTime = Duration.millis(maxReadTime);
}
}

public void setInitialPositionInStream(@Nullable String initialPositionInStream) {
if (initialPositionInStream != null) {
this.initialPositionInStream = InitialPositionInStream.valueOf(initialPositionInStream);
}
}

public void setInitialTimestampInStream(@Nullable Long initialTimestampInStream) {
if (initialTimestampInStream != null) {
this.initialTimestampInStream = Instant.ofEpochMilli(initialTimestampInStream);
}
}

public void setRequestRecordsLimit(@Nullable Long requestRecordsLimit) {
if (requestRecordsLimit != null) {
this.requestRecordsLimit = requestRecordsLimit.intValue();
}
}

public void setUpToDateThreshold(@Nullable Long upToDateThreshold) {
if (upToDateThreshold != null) {
this.upToDateThreshold = Duration.millis(upToDateThreshold);
}
}

public void setMaxCapacityPerShard(@Nullable Long maxCapacityPerShard) {
this.maxCapacityPerShard = maxCapacityPerShard;
}

public void setWatermarkPolicy(@Nullable String watermarkPolicy) {
if (watermarkPolicy != null) {
this.watermarkPolicy = WatermarkPolicy.valueOf(watermarkPolicy);
}
}

public void setWatermarkIdleDurationThreshold(@Nullable Long watermarkIdleDurationThreshold) {
if (watermarkIdleDurationThreshold != null) {
this.watermarkIdleDurationThreshold = Duration.millis(watermarkIdleDurationThreshold);
}
}

public void setRateLimit(@Nullable Long rateLimit) {
if (rateLimit != null) {
this.rateLimit = Duration.millis(rateLimit);
}
}
}

private enum WatermarkPolicy {
ARRIVAL_TIME,
PROCESSING_TIME
}

@Override
public PTransform<PBegin, PCollection<byte[]>> buildExternal(
ReadDataBuilder.Configuration configuration) {
AwsBasicCredentials creds =
AwsBasicCredentials.create(configuration.awsAccessKey, configuration.awsSecretKey);
StaticCredentialsProvider provider = StaticCredentialsProvider.create(creds);
@Nullable URI endpoint = null;
if (configuration.serviceEndpoint != null) {
try {
endpoint = new URI(configuration.serviceEndpoint);
} catch (URISyntaxException ex) {
throw new RuntimeException(
String.format(
"Service endpoint must be URI format, got: %s", configuration.serviceEndpoint));
}
}
KinesisIO.Read readTransform =
KinesisIO.read()
.withStreamName(configuration.streamName)
.withClientConfiguration(
ClientConfiguration.builder()
.credentialsProvider(provider)
.region(Region.of(configuration.region))
.endpoint(endpoint)
.build());

if (configuration.maxNumRecords != null) {
readTransform = readTransform.withMaxNumRecords(configuration.maxNumRecords);
}
if (configuration.upToDateThreshold != null) {
readTransform = readTransform.withUpToDateThreshold(configuration.upToDateThreshold);
}
if (configuration.maxCapacityPerShard != null) {
readTransform =
readTransform.withMaxCapacityPerShard(configuration.maxCapacityPerShard.intValue());
}
if (configuration.watermarkPolicy != null) {
switch (configuration.watermarkPolicy) {
case ARRIVAL_TIME:
readTransform =
configuration.watermarkIdleDurationThreshold != null
? readTransform.withArrivalTimeWatermarkPolicy(
configuration.watermarkIdleDurationThreshold)
: readTransform.withArrivalTimeWatermarkPolicy();
break;
case PROCESSING_TIME:
readTransform = readTransform.withProcessingTimeWatermarkPolicy();
break;
default:
throw new RuntimeException(
String.format(
"Unsupported watermark policy type: %s", configuration.watermarkPolicy));
}
}
if (configuration.rateLimit != null) {
readTransform = readTransform.withFixedDelayRateLimitPolicy(configuration.rateLimit);
}
if (configuration.maxReadTime != null) {
readTransform = readTransform.withMaxReadTime(configuration.maxReadTime);
}
if (configuration.initialPositionInStream != null) {
readTransform =
readTransform.withInitialPositionInStream(configuration.initialPositionInStream);
}
if (configuration.requestRecordsLimit != null) {
readTransform = readTransform.withRequestRecordsLimit(configuration.requestRecordsLimit);
}
if (configuration.initialTimestampInStream != null) {
readTransform =
readTransform.withInitialTimestampInStream(configuration.initialTimestampInStream);
}

return new KinesisReadToBytes(readTransform);
}
}

public static class KinesisReadToBytes extends PTransform<PBegin, PCollection<byte[]>> {
private KinesisIO.Read readTransform;

private KinesisReadToBytes(KinesisIO.Read readTransform) {
this.readTransform = readTransform;
}

@Override
public PCollection<byte[]> expand(PBegin input) {
// Convert back to bytes to keep consistency with previous verison:
// https://github.com/apache/beam/blob/5eed396caf9e0065d8ed82edcc236bad5b71ba22/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisTransformRegistrar.java
return input
.apply(this.readTransform)
.apply(
"Convert to bytes",
ParDo.of(
new DoFn<KinesisRecord, byte[]>() {
@ProcessElement
public byte[] processElement(ProcessContext c) {
KinesisRecord record = c.element();
return record.getDataAsBytes();
}
}));
}
}
}
Loading
Loading