diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index 3f7a4aa139c0..7745ac7153bf 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -421,6 +421,7 @@ class BeamModulePlugin implements Plugin { aws_java_sdk2_auth : "software.amazon.awssdk:auth:$aws_java_sdk2_version", aws_java_sdk2_cloudwatch : "software.amazon.awssdk:cloudwatch:$aws_java_sdk2_version", aws_java_sdk2_dynamodb : "software.amazon.awssdk:dynamodb:$aws_java_sdk2_version", + aws_java_sdk2_kinesis : "software.amazon.awssdk:kinesis:$aws_java_sdk2_version", aws_java_sdk2_sdk_core : "software.amazon.awssdk:sdk-core:$aws_java_sdk2_version", aws_java_sdk2_sns : "software.amazon.awssdk:sns:$aws_java_sdk2_version", bigdataoss_gcsio : "com.google.cloud.bigdataoss:gcsio:$google_cloud_bigdataoss_version", diff --git a/sdks/java/io/amazon-web-services2/build.gradle b/sdks/java/io/amazon-web-services2/build.gradle index eb33c565f052..b1c26474509c 100644 --- a/sdks/java/io/amazon-web-services2/build.gradle +++ b/sdks/java/io/amazon-web-services2/build.gradle @@ -20,6 +20,8 @@ import groovy.json.JsonOutput plugins { id 'org.apache.beam.module' } applyJavaNature(automaticModuleName: 'org.apache.beam.sdk.io.aws2') +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() description = "Apache Beam :: SDKs :: Java :: IO :: Amazon Web Services 2" ext.summary = "IO library to read and write Amazon Web Services services from Beam." @@ -31,15 +33,28 @@ dependencies { compile library.java.aws_java_sdk2_auth compile library.java.aws_java_sdk2_cloudwatch compile library.java.aws_java_sdk2_dynamodb + compile library.java.aws_java_sdk2_kinesis compile library.java.aws_java_sdk2_sdk_core compile library.java.aws_java_sdk2_sns compile library.java.jackson_core compile library.java.jackson_annotations compile library.java.jackson_databind + compile library.java.jackson_dataformat_cbor + compile library.java.joda_time compile library.java.slf4j_api + compile "software.amazon.kinesis:amazon-kinesis-client:2.2.5" + compile "commons-lang:commons-lang:2.6" testCompile project(path: ":sdks:java:core", configuration: "shadowTest") + testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime") + testCompile project(path: ":sdks:java:io:kinesis", configuration: "testRuntime") + testCompile library.java.mockito_core + testCompile library.java.guava_testlib testCompile library.java.hamcrest_core testCompile library.java.junit + testCompile library.java.hamcrest_library + testCompile library.java.powermock + testCompile library.java.powermock_mockito + testCompile "org.assertj:assertj-core:3.11.1" testCompile 'org.testcontainers:testcontainers:1.11.3' testRuntimeOnly library.java.slf4j_jdk14 testRuntimeOnly project(":runners:direct-java") @@ -50,4 +65,6 @@ test { '--region=us-west-2', '--awsCredentialsProvider={"@type": "StaticCredentialsProvider", "accessKeyId": "key_id_value", "secretAccessKey": "secret_value"}' ]) + // Forking every test resolves an issue where KinesisMockReadTest gets stuck forever. + forkEvery 1 } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/AWSClientsProvider.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/AWSClientsProvider.java new file mode 100644 index 000000000000..372de747767e --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/AWSClientsProvider.java @@ -0,0 +1,37 @@ +/* + * 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 java.io.Serializable; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisClient; + +/** + * Provides instances of AWS clients. + * + *

Please note, that any instance of {@link AWSClientsProvider} must be {@link Serializable} to + * ensure it can be sent to worker machines. + */ +public interface AWSClientsProvider extends Serializable { + KinesisClient getKinesisClient(); + + KinesisAsyncClient getKinesisAsyncClient(); + + CloudWatchClient getCloudWatchClient(); +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/BasicKinesisProvider.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/BasicKinesisProvider.java new file mode 100644 index 000000000000..6d130cfe45e7 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/BasicKinesisProvider.java @@ -0,0 +1,92 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import java.net.URI; +import javax.annotation.Nullable; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClientBuilder; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder; +import software.amazon.awssdk.services.kinesis.KinesisClient; +import software.amazon.awssdk.services.kinesis.KinesisClientBuilder; + +/** Basic implementation of {@link AWSClientsProvider} used by default in {@link KinesisIO}. */ +class BasicKinesisProvider implements AWSClientsProvider { + private final String accessKey; + private final String secretKey; + private final String region; + @Nullable private final String serviceEndpoint; + + BasicKinesisProvider( + String accessKey, String secretKey, Region region, @Nullable String serviceEndpoint) { + checkArgument(accessKey != null, "accessKey can not be null"); + checkArgument(secretKey != null, "secretKey can not be null"); + checkArgument(region != null, "region can not be null"); + this.accessKey = accessKey; + this.secretKey = secretKey; + this.region = region.toString(); + this.serviceEndpoint = serviceEndpoint; + } + + private AwsCredentialsProvider getCredentialsProvider() { + return StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKey, secretKey)); + } + + @Override + public KinesisClient getKinesisClient() { + KinesisClientBuilder clientBuilder = + KinesisClient.builder() + .credentialsProvider(getCredentialsProvider()) + .region(Region.of(region)); + if (serviceEndpoint != null) { + clientBuilder.endpointOverride(URI.create(serviceEndpoint)); + } + return clientBuilder.build(); + } + + @Override + public KinesisAsyncClient getKinesisAsyncClient() { + KinesisAsyncClientBuilder clientBuilder = + KinesisAsyncClient.builder() + .credentialsProvider(getCredentialsProvider()) + .region(Region.of(region)); + if (serviceEndpoint != null) { + clientBuilder.endpointOverride(URI.create(serviceEndpoint)); + } + return clientBuilder.build(); + } + + @Override + public CloudWatchClient getCloudWatchClient() { + CloudWatchClientBuilder clientBuilder = + CloudWatchClient.builder() + .credentialsProvider(getCredentialsProvider()) + .region(Region.of(region)); + if (serviceEndpoint != null) { + clientBuilder.endpointOverride(URI.create(serviceEndpoint)); + } + return clientBuilder.build(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CheckpointGenerator.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CheckpointGenerator.java new file mode 100644 index 000000000000..273e509d55a3 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CheckpointGenerator.java @@ -0,0 +1,29 @@ +/* + * 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 java.io.Serializable; + +/** + * Used to generate checkpoint object on demand. How exactly the checkpoint is generated is up to + * implementing class. + */ +interface CheckpointGenerator extends Serializable { + + KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) throws TransientKinesisException; +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptional.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptional.java new file mode 100644 index 000000000000..2eae59aab65a --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptional.java @@ -0,0 +1,101 @@ +/* + * 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 java.util.NoSuchElementException; +import java.util.Objects; + +/** + * Similar to Guava {@code Optional}, but throws {@link NoSuchElementException} for missing element. + */ +abstract class CustomOptional { + + @SuppressWarnings("unchecked") + public static CustomOptional absent() { + return (Absent) Absent.INSTANCE; + } + + public static CustomOptional of(T v) { + return new Present<>(v); + } + + public abstract boolean isPresent(); + + public abstract T get(); + + private static class Present extends CustomOptional { + + private final T value; + + private Present(T value) { + this.value = value; + } + + @Override + public boolean isPresent() { + return true; + } + + @Override + public T get() { + return value; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof Present)) { + return false; + } + + Present present = (Present) o; + return Objects.equals(value, present.value); + } + + @Override + public int hashCode() { + return Objects.hash(value); + } + } + + private static class Absent extends CustomOptional { + + private static final Absent INSTANCE = new Absent<>(); + + private Absent() {} + + @Override + public boolean isPresent() { + return false; + } + + @Override + public T get() { + throw new NoSuchElementException(); + } + + @Override + public boolean equals(Object o) { + return o instanceof Absent; + } + + @Override + public int hashCode() { + return 0; + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGenerator.java new file mode 100644 index 000000000000..bfef2498f798 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGenerator.java @@ -0,0 +1,81 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.Set; +import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.kinesis.model.Shard; + +/** + * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream. List of + * shards is obtained dynamically on call to {@link #generate(SimplifiedKinesisClient)}. + */ +class DynamicCheckpointGenerator implements CheckpointGenerator { + + private static final Logger LOG = LoggerFactory.getLogger(DynamicCheckpointGenerator.class); + private final String streamName; + private final String consumerArn; + private final StartingPoint startingPoint; + private final StartingPointShardsFinder startingPointShardsFinder; + + public DynamicCheckpointGenerator( + String streamName, String consumerArn, StartingPoint startingPoint) { + this.streamName = streamName; + this.consumerArn = consumerArn; + this.startingPoint = startingPoint; + this.startingPointShardsFinder = new StartingPointShardsFinder(); + } + + public DynamicCheckpointGenerator( + String streamName, + String consumerArn, + StartingPoint startingPoint, + StartingPointShardsFinder startingPointShardsFinder) { + this.streamName = checkNotNull(streamName, "streamName"); + this.consumerArn = consumerArn; + this.startingPoint = checkNotNull(startingPoint, "startingPoint"); + this.startingPointShardsFinder = + checkNotNull(startingPointShardsFinder, "startingPointShardsFinder"); + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient kinesis) + throws TransientKinesisException { + Set shardsAtStartingPoint = + startingPointShardsFinder.findShardsAtStartingPoint(kinesis, streamName, startingPoint); + LOG.info( + "Creating a checkpoint with following shards {} at {}", + shardsAtStartingPoint, + startingPoint.getTimestamp()); + return new KinesisReaderCheckpoint( + shardsAtStartingPoint.stream() + .map( + shard -> + new ShardCheckpoint(streamName, shard.shardId(), consumerArn, startingPoint)) + .collect(Collectors.toList())); + } + + @Override + public String toString() { + return String.format("Checkpoint generator for %s: %s", streamName, startingPoint); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/GetKinesisRecordsResult.java new file mode 100644 index 000000000000..8cf47ba2bdc6 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/GetKinesisRecordsResult.java @@ -0,0 +1,60 @@ +/* + * 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 java.util.List; +import java.util.stream.Collectors; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +/** Represents the output of 'get' operation on Kinesis stream. */ +class GetKinesisRecordsResult { + + private final List records; + private final String nextShardIterator; + private final long millisBehindLatest; + + public GetKinesisRecordsResult( + List records, + String nextShardIterator, + long millisBehindLatest, + final String streamName, + final String shardId) { + this.records = + records.stream() + .map( + input -> { + assert input != null; // to make FindBugs happy + return new KinesisRecord(input, streamName, shardId); + }) + .collect(Collectors.toList()); + this.nextShardIterator = nextShardIterator; + this.millisBehindLatest = millisBehindLatest; + } + + public List getRecords() { + return records; + } + + public String getNextShardIterator() { + return nextShardIterator; + } + + public long getMillisBehindLatest() { + return millisBehindLatest; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java new file mode 100644 index 000000000000..b37cf93a3f95 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java @@ -0,0 +1,378 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.io.Read.Unbounded; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.kinesis.KinesisClient; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** + * {@link PTransform}s for reading from and writing to Kinesis streams. + * + *

Reading from Kinesis

+ * + *

Example usage: + * + *

{@code
+ * p.apply(KinesisIO.read()
+ *     .withStreamName("streamName")
+ *     .withInitialPositionInStream(InitialPositionInStream.LATEST)
+ *     .withAWSClientsProvider("AWS_KEY", _"AWS_SECRET", STREAM_REGION)
+ *  .apply( ... ) // other transformations
+ * }
+ * + *

As you can see you need to provide 3 things: + * + *

    + *
  • name of the stream you're going to read + *
  • position in the stream where reading should start. There are two options: + *
      + *
    • {@link InitialPositionInStream#LATEST} - reading will begin from end of the stream + *
    • {@link InitialPositionInStream#TRIM_HORIZON} - reading will begin at the very + * beginning of the stream + *
    + *
  • data used to initialize {@link KinesisClient} and {@link CloudWatchClient} clients: + *
      + *
    • credentials (aws key, aws secret) + *
    • region where the stream is located + *
    + *
+ * + *

In case when you want to set up {@link KinesisClient} or {@link CloudWatchClient} client by + * your own (for example if you're using more sophisticated authorization methods like Amazon STS, + * etc.) you can do it by implementing {@link AWSClientsProvider} class: + * + *

{@code
+ * public class MyCustomKinesisClientProvider implements AWSClientsProvider {
+ *   {@literal @}Override
+ *   public KinesisClient getKinesisClient() {
+ *     // set up your client here
+ *   }
+ *
+ *   public CloudWatchClient getCloudWatchClient() {
+ *     // set up your client here
+ *   }
+ *
+ * }
+ * }
+ * + *

Usage is pretty straightforward: + * + *

{@code
+ * p.apply(KinesisIO.read()
+ *    .withStreamName("streamName")
+ *    .withInitialPositionInStream(InitialPositionInStream.LATEST)
+ *    .withAWSClientsProvider(new MyCustomKinesisClientProvider())
+ *  .apply( ... ) // other transformations
+ * }
+ * + *

There’s also possibility to start reading using arbitrary point in time - in this case you + * need to provide {@link Instant} object: + * + *

{@code
+ * p.apply(KinesisIO.read()
+ *     .withStreamName("streamName")
+ *     .withInitialTimestampInStream(instant)
+ *     .withAWSClientsProvider(new MyCustomKinesisClientProvider())
+ *  .apply( ... ) // other transformations
+ * }
+ * + *

Kinesis IO uses ArrivalTimeWatermarkPolicy by default. To use Processing time as event time: + * + *

{@code
+ * p.apply(KinesisIO.read()
+ *    .withStreamName("streamName")
+ *    .withInitialPositionInStream(InitialPositionInStream.LATEST)
+ *    .withProcessingTimeWatermarkPolicy())
+ * }
+ * + *

It is also possible to specify a custom watermark policy to control watermark computation. + * Below is an example + * + *

{@code
+ * // custom policy
+ * class MyCustomPolicy implements WatermarkPolicy {
+ *     private WatermarkPolicyFactory.CustomWatermarkPolicy customWatermarkPolicy;
+ *
+ *     MyCustomPolicy() {
+ *       this.customWatermarkPolicy = new WatermarkPolicyFactory.CustomWatermarkPolicy(WatermarkParameters.create());
+ *     }
+ *
+ *     @Override
+ *     public Instant getWatermark() {
+ *       return customWatermarkPolicy.getWatermark();
+ *     }
+ *
+ *     @Override
+ *     public void update(KinesisRecord record) {
+ *       customWatermarkPolicy.update(record);
+ *     }
+ *   }
+ *
+ * // custom factory
+ * class MyCustomPolicyFactory implements WatermarkPolicyFactory {
+ *     @Override
+ *     public WatermarkPolicy createWatermarkPolicy() {
+ *       return new MyCustomPolicy();
+ *     }
+ * }
+ *
+ * p.apply(KinesisIO.read()
+ *    .withStreamName("streamName")
+ *    .withInitialPositionInStream(InitialPositionInStream.LATEST)
+ *    .withCustomWatermarkPolicy(new MyCustomPolicyFactory())
+ * }
+ */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public final class KinesisIO { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisIO.class); + + private static final int DEFAULT_NUM_RETRIES = 6; + + /** Returns a new {@link Read} transform for reading from Kinesis. */ + public static Read read() { + return new AutoValue_KinesisIO_Read.Builder() + .setMaxNumRecords(Long.MAX_VALUE) + .setUpToDateThreshold(Duration.ZERO) + .setWatermarkPolicyFactory(WatermarkPolicyFactory.withArrivalTimePolicy()) + .build(); + } + + /** Implementation of {@link #read}. */ + @AutoValue + public abstract static class Read extends PTransform> { + + @Nullable + abstract String getStreamName(); + + @Nullable + abstract String getConsumerArn(); + + @Nullable + abstract StartingPoint getInitialPosition(); + + @Nullable + abstract AWSClientsProvider getAWSClientsProvider(); + + abstract long getMaxNumRecords(); + + @Nullable + abstract Duration getMaxReadTime(); + + abstract Duration getUpToDateThreshold(); + + @Nullable + abstract Integer getRequestRecordsLimit(); + + abstract WatermarkPolicyFactory getWatermarkPolicyFactory(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + abstract Builder setStreamName(String streamName); + + abstract Builder setConsumerArn(String consumerArn); + + abstract Builder setInitialPosition(StartingPoint startingPoint); + + abstract Builder setAWSClientsProvider(AWSClientsProvider clientProvider); + + abstract Builder setMaxNumRecords(long maxNumRecords); + + abstract Builder setMaxReadTime(Duration maxReadTime); + + abstract Builder setUpToDateThreshold(Duration upToDateThreshold); + + abstract Builder setRequestRecordsLimit(Integer limit); + + abstract Builder setWatermarkPolicyFactory(WatermarkPolicyFactory watermarkPolicyFactory); + + abstract Read build(); + } + + /** Specify reading from streamName. */ + public Read withStreamName(String streamName) { + return toBuilder().setStreamName(streamName).build(); + } + + /** Specify reading from an enhanced fan-out consumer. */ + public Read withConsumerArn(String consumerArn) { + return toBuilder().setConsumerArn(consumerArn).build(); + } + + /** Specify reading from some initial position in stream. */ + public Read withInitialPositionInStream(InitialPositionInStream initialPosition) { + return toBuilder().setInitialPosition(new StartingPoint(initialPosition)).build(); + } + + /** + * Specify reading beginning at given {@link Instant}. This {@link Instant} must be in the past, + * i.e. before {@link Instant#now()}. + */ + public Read withInitialTimestampInStream(Instant initialTimestamp) { + return toBuilder().setInitialPosition(new StartingPoint(initialTimestamp)).build(); + } + + /** + * Allows to specify custom {@link AWSClientsProvider}. {@link AWSClientsProvider provides + * {@link KinesisClient} and {@link CloudWatchClient} instances which are later used for + * communication with Kinesis. You should use this method if {@link + * Read#withAWSClientsProvider(String, String, Region)} does not suit your needs. + */ + public Read withAWSClientsProvider(AWSClientsProvider awsClientsProvider) { + return toBuilder().setAWSClientsProvider(awsClientsProvider).build(); + } + + /** + * Specify credential details and region to be used to read from Kinesis. If you need more + * sophisticated credential protocol, then you should look at {@link + * Read#withAWSClientsProvider(AWSClientsProvider)}. + */ + public Read withAWSClientsProvider(String awsAccessKey, String awsSecretKey, Region region) { + return withAWSClientsProvider(awsAccessKey, awsSecretKey, region, null); + } + + /** + * Specify credential details and region to be used to read from Kinesis. If you need more + * sophisticated credential protocol, then you should look at {@link + * Read#withAWSClientsProvider(AWSClientsProvider)}. + * + *

The {@code serviceEndpoint} sets an alternative service host. This is useful to execute + * the tests with a kinesis service emulator. + */ + public Read withAWSClientsProvider( + String awsAccessKey, String awsSecretKey, Region region, String serviceEndpoint) { + return withAWSClientsProvider( + new BasicKinesisProvider(awsAccessKey, awsSecretKey, region, serviceEndpoint)); + } + + /** Specifies to read at most a given number of records. */ + public Read withMaxNumRecords(long maxNumRecords) { + checkArgument( + maxNumRecords > 0, "maxNumRecords must be positive, but was: %s", maxNumRecords); + return toBuilder().setMaxNumRecords(maxNumRecords).build(); + } + + /** Specifies to read records during {@code maxReadTime}. */ + public Read withMaxReadTime(Duration maxReadTime) { + checkArgument(maxReadTime != null, "maxReadTime can not be null"); + return toBuilder().setMaxReadTime(maxReadTime).build(); + } + + /** + * Specifies how late records consumed by this source can be to still be considered on time. + * When this limit is exceeded the actual backlog size will be evaluated and the runner might + * decide to scale the amount of resources allocated to the pipeline in order to speed up + * ingestion. + */ + public Read withUpToDateThreshold(Duration upToDateThreshold) { + checkArgument(upToDateThreshold != null, "upToDateThreshold can not be null"); + return toBuilder().setUpToDateThreshold(upToDateThreshold).build(); + } + + /** + * Specifies the maximum number of records in GetRecordsResult returned by GetRecords call which + * is limited by 10K records. If should be adjusted according to average size of data record to + * prevent shard overloading. More details can be found here: API_GetRecords + */ + public Read withRequestRecordsLimit(int limit) { + checkArgument(limit > 0, "limit must be positive, but was: %s", limit); + checkArgument(limit <= 10_000, "limit must be up to 10,000, but was: %s", limit); + return toBuilder().setRequestRecordsLimit(limit).build(); + } + + /** Specifies the {@code WatermarkPolicyFactory} as ArrivalTimeWatermarkPolicyFactory. */ + public Read withArrivalTimeWatermarkPolicy() { + return toBuilder() + .setWatermarkPolicyFactory(WatermarkPolicyFactory.withArrivalTimePolicy()) + .build(); + } + + /** + * Specifies the {@code WatermarkPolicyFactory} as ArrivalTimeWatermarkPolicyFactory. + * + *

{@param watermarkIdleDurationThreshold} Denotes the duration for which the watermark can + * be idle. + */ + public Read withArrivalTimeWatermarkPolicy(Duration watermarkIdleDurationThreshold) { + return toBuilder() + .setWatermarkPolicyFactory( + WatermarkPolicyFactory.withArrivalTimePolicy(watermarkIdleDurationThreshold)) + .build(); + } + + /** Specifies the {@code WatermarkPolicyFactory} as ProcessingTimeWatermarkPolicyFactory. */ + public Read withProcessingTimeWatermarkPolicy() { + return toBuilder() + .setWatermarkPolicyFactory(WatermarkPolicyFactory.withProcessingTimePolicy()) + .build(); + } + + /** + * Specifies the {@code WatermarkPolicyFactory} as a custom watermarkPolicyFactory. + * + * @param watermarkPolicyFactory Custom Watermark policy factory. + */ + public Read withCustomWatermarkPolicy(WatermarkPolicyFactory watermarkPolicyFactory) { + checkArgument(watermarkPolicyFactory != null, "watermarkPolicyFactory cannot be null"); + return toBuilder().setWatermarkPolicyFactory(watermarkPolicyFactory).build(); + } + + @Override + public PCollection expand(PBegin input) { + Unbounded unbounded = + org.apache.beam.sdk.io.Read.from( + new KinesisSource( + getAWSClientsProvider(), + getStreamName(), + getConsumerArn(), + getInitialPosition(), + getUpToDateThreshold(), + getWatermarkPolicyFactory(), + getRequestRecordsLimit())); + + PTransform> transform = unbounded; + + if (getMaxNumRecords() < Long.MAX_VALUE || getMaxReadTime() != null) { + transform = + unbounded.withMaxReadTime(getMaxReadTime()).withMaxNumRecords(getMaxNumRecords()); + } + + return input.apply(transform); + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java new file mode 100644 index 000000000000..46ebea642554 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java @@ -0,0 +1,182 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Reads data from multiple kinesis shards in a single thread. It uses simple round robin algorithm + * when fetching data from shards. + */ +class KinesisReader extends UnboundedSource.UnboundedReader { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisReader.class); + + private final SimplifiedKinesisClient kinesis; + private final KinesisSource source; + private final CheckpointGenerator initialCheckpointGenerator; + private final WatermarkPolicyFactory watermarkPolicyFactory; + private final Duration upToDateThreshold; + private final Duration backlogBytesCheckThreshold; + private CustomOptional currentRecord = CustomOptional.absent(); + private long lastBacklogBytes; + private Instant backlogBytesLastCheckTime = new Instant(0L); + private ShardReadersPool shardReadersPool; + + KinesisReader( + SimplifiedKinesisClient kinesis, + CheckpointGenerator initialCheckpointGenerator, + KinesisSource source, + WatermarkPolicyFactory watermarkPolicyFactory, + Duration upToDateThreshold) { + this( + kinesis, + initialCheckpointGenerator, + source, + watermarkPolicyFactory, + upToDateThreshold, + Duration.standardSeconds(30)); + } + + KinesisReader( + SimplifiedKinesisClient kinesis, + CheckpointGenerator initialCheckpointGenerator, + KinesisSource source, + WatermarkPolicyFactory watermarkPolicyFactory, + Duration upToDateThreshold, + Duration backlogBytesCheckThreshold) { + this.kinesis = checkNotNull(kinesis, "kinesis"); + this.initialCheckpointGenerator = + checkNotNull(initialCheckpointGenerator, "initialCheckpointGenerator"); + this.watermarkPolicyFactory = watermarkPolicyFactory; + this.source = source; + this.upToDateThreshold = upToDateThreshold; + this.backlogBytesCheckThreshold = backlogBytesCheckThreshold; + } + + /** Generates initial checkpoint and instantiates iterators for shards. */ + @Override + public boolean start() throws IOException { + LOG.info("Starting reader using {}", initialCheckpointGenerator); + + try { + shardReadersPool = createShardReadersPool(); + shardReadersPool.start(); + } catch (TransientKinesisException e) { + throw new IOException(e); + } + + return advance(); + } + + /** Retrieves next record from internal buffer. */ + @Override + public boolean advance() throws IOException { + currentRecord = shardReadersPool.nextRecord(); + return currentRecord.isPresent(); + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return currentRecord.get().getUniqueId(); + } + + @Override + public KinesisRecord getCurrent() throws NoSuchElementException { + return currentRecord.get(); + } + + /** + * Returns the approximate time that the current record was inserted into the stream. It is not + * guaranteed to be accurate - this could lead to mark some records as "late" even if they were + * not. Beware of this when setting {@link + * org.apache.beam.sdk.values.WindowingStrategy#withAllowedLateness} + */ + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentRecord.get().getApproximateArrivalTimestamp(); + } + + @Override + public void close() throws IOException { + shardReadersPool.stop(); + } + + @Override + public Instant getWatermark() { + return shardReadersPool.getWatermark(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + return shardReadersPool.getCheckpointMark(); + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + + /** + * Returns total size of all records that remain in Kinesis stream after current watermark. If the + * watermark was not already set then it returns {@link + * UnboundedSource.UnboundedReader#BACKLOG_UNKNOWN}. When currently processed record is not + * further behind than {@link #upToDateThreshold} then this method returns 0. + */ + @Override + public long getTotalBacklogBytes() { + Instant watermark = getWatermark(); + + if (watermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) { + return UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN; + } + + if (watermark.plus(upToDateThreshold).isAfterNow()) { + return 0L; + } + if (backlogBytesLastCheckTime.plus(backlogBytesCheckThreshold).isAfterNow()) { + return lastBacklogBytes; + } + try { + lastBacklogBytes = kinesis.getBacklogBytes(source.getStreamName(), watermark); + backlogBytesLastCheckTime = Instant.now(); + } catch (TransientKinesisException e) { + LOG.warn("Transient exception occurred.", e); + } + LOG.info( + "Total backlog bytes for {} stream with {} watermark: {}", + source.getStreamName(), + watermark, + lastBacklogBytes); + return lastBacklogBytes; + } + + ShardReadersPool createShardReadersPool() throws TransientKinesisException { + return new ShardReadersPool( + kinesis, initialCheckpointGenerator.generate(kinesis), watermarkPolicyFactory); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpoint.java new file mode 100644 index 000000000000..aba80eccff02 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpoint.java @@ -0,0 +1,76 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.newArrayList; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.partition; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; + +/** + * Checkpoint representing a total progress in a set of shards in single stream. The set of shards + * covered by {@link KinesisReaderCheckpoint} may or may not be equal to set of all shards present + * in the stream. This class is immutable. + */ +class KinesisReaderCheckpoint + implements Iterable, UnboundedSource.CheckpointMark, Serializable { + + private final List shardCheckpoints; + + public KinesisReaderCheckpoint(Iterable shardCheckpoints) { + this.shardCheckpoints = ImmutableList.copyOf(shardCheckpoints); + } + + /** + * Splits given multi-shard checkpoint into partitions of approximately equal size. + * + * @param desiredNumSplits - upper limit for number of partitions to generate. + * @return list of checkpoints covering consecutive partitions of current checkpoint. + */ + public List splitInto(int desiredNumSplits) { + int partitionSize = divideAndRoundUp(shardCheckpoints.size(), desiredNumSplits); + + List checkpoints = newArrayList(); + for (List shardPartition : partition(shardCheckpoints, partitionSize)) { + checkpoints.add(new KinesisReaderCheckpoint(shardPartition)); + } + return checkpoints; + } + + private int divideAndRoundUp(int nominator, int denominator) { + return (nominator + denominator - 1) / denominator; + } + + @Override + public void finalizeCheckpoint() throws IOException {} + + @Override + public String toString() { + return shardCheckpoints.toString(); + } + + @Override + public Iterator iterator() { + return shardCheckpoints.iterator(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java new file mode 100644 index 000000000000..055f74b5046a --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java @@ -0,0 +1,142 @@ +/* + * 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 static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.apache.commons.lang.builder.ToStringBuilder; +import org.joda.time.Instant; +import software.amazon.kinesis.retrieval.KinesisClientRecord; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +/** {@link KinesisClientRecord} enhanced with utility methods. */ +public class KinesisRecord { + + private Instant readTime; + private String streamName; + private String shardId; + private long subSequenceNumber; + private String sequenceNumber; + private Instant approximateArrivalTimestamp; + private ByteBuffer data; + private String partitionKey; + + public KinesisRecord(KinesisClientRecord record, String streamName, String shardId) { + this( + record.data(), + record.sequenceNumber(), + record.subSequenceNumber(), + record.partitionKey(), + TimeUtil.toJoda(record.approximateArrivalTimestamp()), + Instant.now(), + streamName, + shardId); + } + + public KinesisRecord( + ByteBuffer data, + String sequenceNumber, + long subSequenceNumber, + String partitionKey, + Instant approximateArrivalTimestamp, + Instant readTime, + String streamName, + String shardId) { + this.data = copyData(data); + this.sequenceNumber = sequenceNumber; + this.subSequenceNumber = subSequenceNumber; + this.partitionKey = partitionKey; + this.approximateArrivalTimestamp = approximateArrivalTimestamp; + this.readTime = readTime; + this.streamName = streamName; + this.shardId = shardId; + } + + private ByteBuffer copyData(ByteBuffer data) { + data.rewind(); + byte[] bytes = new byte[data.remaining()]; + data.get(bytes); + return ByteBuffer.wrap(bytes); + } + + public ExtendedSequenceNumber getExtendedSequenceNumber() { + return new ExtendedSequenceNumber(getSequenceNumber(), getSubSequenceNumber()); + } + + /** @return The unique identifier of the record based on its position in the stream. */ + public byte[] getUniqueId() { + return getExtendedSequenceNumber().toString().getBytes(StandardCharsets.UTF_8); + } + + public Instant getReadTime() { + return readTime; + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } + + public byte[] getDataAsBytes() { + return getData().array(); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + + @Override + public String toString() { + return ToStringBuilder.reflectionToString(this); + } + + public long getSubSequenceNumber() { + return subSequenceNumber; + } + + /** @return The unique identifier of the record within its shard. */ + public String getSequenceNumber() { + return sequenceNumber; + } + + /** @return The approximate time that the record was inserted into the stream. */ + public Instant getApproximateArrivalTimestamp() { + return approximateArrivalTimestamp; + } + + /** @return The data blob. */ + public ByteBuffer getData() { + return data; + } + + public String getPartitionKey() { + return partitionKey; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoder.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoder.java new file mode 100644 index 000000000000..1bad1e4edd23 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoder.java @@ -0,0 +1,76 @@ +/* + * 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 java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.joda.time.Instant; + +/** A {@link Coder} for {@link KinesisRecord}. */ +class KinesisRecordCoder extends AtomicCoder { + + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + private static final ByteArrayCoder BYTE_ARRAY_CODER = ByteArrayCoder.of(); + private static final InstantCoder INSTANT_CODER = InstantCoder.of(); + private static final VarLongCoder VAR_LONG_CODER = VarLongCoder.of(); + + public static KinesisRecordCoder of() { + return new KinesisRecordCoder(); + } + + @Override + public void encode(KinesisRecord value, OutputStream outStream) throws IOException { + BYTE_ARRAY_CODER.encode(value.getDataAsBytes(), outStream); + STRING_CODER.encode(value.getSequenceNumber(), outStream); + STRING_CODER.encode(value.getPartitionKey(), outStream); + INSTANT_CODER.encode(value.getApproximateArrivalTimestamp(), outStream); + VAR_LONG_CODER.encode(value.getSubSequenceNumber(), outStream); + INSTANT_CODER.encode(value.getReadTime(), outStream); + STRING_CODER.encode(value.getStreamName(), outStream); + STRING_CODER.encode(value.getShardId(), outStream); + } + + @Override + public KinesisRecord decode(InputStream inStream) throws IOException { + ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream)); + String sequenceNumber = STRING_CODER.decode(inStream); + String partitionKey = STRING_CODER.decode(inStream); + Instant approximateArrivalTimestamp = INSTANT_CODER.decode(inStream); + long subSequenceNumber = VAR_LONG_CODER.decode(inStream); + Instant readTimestamp = INSTANT_CODER.decode(inStream); + String streamName = STRING_CODER.decode(inStream); + String shardId = STRING_CODER.decode(inStream); + return new KinesisRecord( + data, + sequenceNumber, + subSequenceNumber, + partitionKey, + approximateArrivalTimestamp, + readTimestamp, + streamName, + shardId); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisShardClosedException.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisShardClosedException.java new file mode 100644 index 000000000000..0c6e471f7d4e --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisShardClosedException.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** Internal exception thrown when shard end is encountered during iteration. */ +class KinesisShardClosedException extends Exception { + + KinesisShardClosedException(String message) { + super(message); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java new file mode 100644 index 000000000000..6e888cc62970 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java @@ -0,0 +1,151 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.newArrayList; + +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Represents source for single stream in Kinesis. */ +class KinesisSource extends UnboundedSource { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + + private final AWSClientsProvider awsClientsProvider; + private final String streamName; + private final String consumerArn; + private final Duration upToDateThreshold; + private final WatermarkPolicyFactory watermarkPolicyFactory; + private CheckpointGenerator initialCheckpointGenerator; + private final Integer limit; + + KinesisSource( + AWSClientsProvider awsClientsProvider, + String streamName, + String consumerArn, + StartingPoint startingPoint, + Duration upToDateThreshold, + WatermarkPolicyFactory watermarkPolicyFactory, + Integer limit) { + this( + awsClientsProvider, + new DynamicCheckpointGenerator(streamName, consumerArn, startingPoint), + streamName, + consumerArn, + upToDateThreshold, + watermarkPolicyFactory, + limit); + } + + private KinesisSource( + AWSClientsProvider awsClientsProvider, + CheckpointGenerator initialCheckpoint, + String streamName, + String consumerArn, + Duration upToDateThreshold, + WatermarkPolicyFactory watermarkPolicyFactory, + Integer limit) { + this.awsClientsProvider = awsClientsProvider; + this.initialCheckpointGenerator = initialCheckpoint; + this.streamName = streamName; + this.consumerArn = consumerArn; + this.upToDateThreshold = upToDateThreshold; + this.watermarkPolicyFactory = watermarkPolicyFactory; + this.limit = limit; + validate(); + } + + /** + * Generate splits for reading from the stream. Basically, it'll try to evenly split set of shards + * in the stream into {@code desiredNumSplits} partitions. Each partition is then a split. + */ + @Override + public List split(int desiredNumSplits, PipelineOptions options) throws Exception { + KinesisReaderCheckpoint checkpoint = + initialCheckpointGenerator.generate( + SimplifiedKinesisClient.from(awsClientsProvider, limit)); + + List sources = newArrayList(); + + for (KinesisReaderCheckpoint partition : checkpoint.splitInto(desiredNumSplits)) { + sources.add( + new KinesisSource( + awsClientsProvider, + new StaticCheckpointGenerator(partition), + streamName, + consumerArn, + upToDateThreshold, + watermarkPolicyFactory, + limit)); + } + return sources; + } + + /** + * Creates reader based on given {@link KinesisReaderCheckpoint}. If {@link + * KinesisReaderCheckpoint} is not given, then we use {@code initialCheckpointGenerator} to + * generate new checkpoint. + */ + @Override + public UnboundedReader createReader( + PipelineOptions options, KinesisReaderCheckpoint checkpointMark) { + + CheckpointGenerator checkpointGenerator = initialCheckpointGenerator; + + if (checkpointMark != null) { + checkpointGenerator = new StaticCheckpointGenerator(checkpointMark); + } + + LOG.info("Creating new reader using {}", checkpointGenerator); + + return new KinesisReader( + SimplifiedKinesisClient.from(awsClientsProvider, limit), + checkpointGenerator, + this, + watermarkPolicyFactory, + upToDateThreshold); + } + + @Override + public Coder getCheckpointMarkCoder() { + return SerializableCoder.of(KinesisReaderCheckpoint.class); + } + + @Override + public void validate() { + checkNotNull(awsClientsProvider); + checkNotNull(initialCheckpointGenerator); + } + + @Override + public Coder getOutputCoder() { + return KinesisRecordCoder.of(); + } + + String getStreamName() { + return streamName; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilter.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilter.java new file mode 100644 index 000000000000..b6f250d27345 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilter.java @@ -0,0 +1,41 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.newArrayList; + +import java.util.List; + +/** + * Filters out records, which were already processed and checkpointed. + * + *

We need this step, because we can get iterators from Kinesis only with "sequenceNumber" + * accuracy, not with "subSequenceNumber" accuracy. + */ +class RecordFilter { + + public List apply(List records, ShardCheckpoint checkpoint) { + List filteredRecords = newArrayList(); + for (KinesisRecord record : records) { + if (checkpoint.isBeforeOrAt(record)) { + filteredRecords.add(record); + } + } + return filteredRecords; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java new file mode 100644 index 000000000000..eb61747f886f --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java @@ -0,0 +1,221 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.LATEST; + +import java.io.Serializable; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import org.joda.time.Instant; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +/** + * Checkpoint mark for single shard in the stream. Current position in the shard is determined by + * either: + * + *

    + *
  • {@link #shardIteratorType} if it is equal to {@link ShardIteratorType#LATEST} or {@link + * ShardIteratorType#TRIM_HORIZON} + *
  • combination of {@link #sequenceNumber} and {@link #subSequenceNumber} if {@link + * ShardIteratorType#AFTER_SEQUENCE_NUMBER} or {@link ShardIteratorType#AT_SEQUENCE_NUMBER} + *
+ * + * This class is immutable. + */ +class ShardCheckpoint implements Serializable { + + private final String streamName; + private final String shardId; + private final String consumerArn; + private final String sequenceNumber; + private final ShardIteratorType shardIteratorType; + private final Long subSequenceNumber; + private final Instant timestamp; + + public ShardCheckpoint( + String streamName, String shardId, String consumerArn, StartingPoint startingPoint) { + this( + streamName, + shardId, + consumerArn, + ShardIteratorType.fromValue(startingPoint.getPositionName()), + startingPoint.getTimestamp()); + } + + public ShardCheckpoint( + String streamName, + String shardId, + String consumerArn, + ShardIteratorType shardIteratorType, + Instant timestamp) { + this(streamName, shardId, consumerArn, shardIteratorType, null, null, timestamp); + } + + public ShardCheckpoint( + String streamName, + String shardId, + String consumerArn, + ShardIteratorType shardIteratorType, + String sequenceNumber, + Long subSequenceNumber) { + this( + streamName, + shardId, + consumerArn, + shardIteratorType, + sequenceNumber, + subSequenceNumber, + null); + } + + private ShardCheckpoint( + String streamName, + String shardId, + String consumerArn, + ShardIteratorType shardIteratorType, + String sequenceNumber, + Long subSequenceNumber, + Instant timestamp) { + this.shardIteratorType = checkNotNull(shardIteratorType, "shardIteratorType"); + this.streamName = checkNotNull(streamName, "streamName"); + this.shardId = checkNotNull(shardId, "shardId"); + this.consumerArn = consumerArn; + if (shardIteratorType == AT_SEQUENCE_NUMBER || shardIteratorType == AFTER_SEQUENCE_NUMBER) { + checkNotNull( + sequenceNumber, + "You must provide sequence number for AT_SEQUENCE_NUMBER" + " or AFTER_SEQUENCE_NUMBER"); + } else { + checkArgument( + sequenceNumber == null, + "Sequence number must be null for LATEST, TRIM_HORIZON or AT_TIMESTAMP"); + } + if (shardIteratorType == AT_TIMESTAMP) { + checkNotNull(timestamp, "You must provide timestamp for AT_TIMESTAMP"); + } else { + checkArgument( + timestamp == null, "Timestamp must be null for an iterator type other than AT_TIMESTAMP"); + } + + this.subSequenceNumber = subSequenceNumber; + this.sequenceNumber = sequenceNumber; + this.timestamp = timestamp; + } + + /** + * Used to compare {@link ShardCheckpoint} object to {@link KinesisRecord}. Depending on the the + * underlying shardIteratorType, it will either compare the timestamp or the {@link + * ExtendedSequenceNumber}. + * + * @param other + * @return if current checkpoint mark points before or at given {@link ExtendedSequenceNumber} + */ + public boolean isBeforeOrAt(KinesisRecord other) { + if (shardIteratorType == AT_TIMESTAMP) { + return timestamp.compareTo(other.getApproximateArrivalTimestamp()) <= 0; + } + int result = extendedSequenceNumber().compareTo(other.getExtendedSequenceNumber()); + if (result == 0) { + return shardIteratorType == AT_SEQUENCE_NUMBER; + } + return result < 0; + } + + private ExtendedSequenceNumber extendedSequenceNumber() { + String fullSequenceNumber = sequenceNumber; + if (fullSequenceNumber == null) { + fullSequenceNumber = shardIteratorType.toString(); + } + return new ExtendedSequenceNumber(fullSequenceNumber, subSequenceNumber); + } + + @Override + public String toString() { + return String.format( + "Checkpoint %s for stream %s, shard %s: %s", + shardIteratorType, streamName, shardId, sequenceNumber); + } + + public CompletableFuture subscribeToShard( + boolean resubscribe, + SimplifiedKinesisClient kinesisClient, + final SubscribeToShardResponseHandler.Visitor visitor, + final Consumer onError) + throws TransientKinesisException { + if (resubscribe) { + return kinesisClient.subscribeToShard( + consumerArn, shardId, LATEST, null, null, visitor, onError); + } + if (checkpointIsInTheMiddleOfAUserRecord()) { + return kinesisClient.subscribeToShard( + consumerArn, shardId, AT_SEQUENCE_NUMBER, sequenceNumber, null, visitor, onError); + } + return kinesisClient.subscribeToShard( + consumerArn, shardId, shardIteratorType, sequenceNumber, timestamp, visitor, onError); + } + + public String getShardIterator(SimplifiedKinesisClient kinesisClient) + throws TransientKinesisException { + if (checkpointIsInTheMiddleOfAUserRecord()) { + return kinesisClient.getShardIterator( + streamName, shardId, AT_SEQUENCE_NUMBER, sequenceNumber, null); + } + return kinesisClient.getShardIterator( + streamName, shardId, shardIteratorType, sequenceNumber, timestamp); + } + + private boolean checkpointIsInTheMiddleOfAUserRecord() { + return shardIteratorType == AFTER_SEQUENCE_NUMBER && subSequenceNumber != null; + } + + /** + * Used to advance checkpoint mark to position after given {@link Record}. + * + * @param record + * @return new checkpoint object pointing directly after given {@link Record} + */ + public ShardCheckpoint moveAfter(KinesisRecord record) { + return new ShardCheckpoint( + streamName, + shardId, + consumerArn, + AFTER_SEQUENCE_NUMBER, + record.getSequenceNumber(), + record.getSubSequenceNumber()); + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } + + public String getConsumerArn() { + return consumerArn; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java new file mode 100644 index 000000000000..7495a471e6b3 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java @@ -0,0 +1,337 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal shard iterators pool. It maintains the thread pool for reading Kinesis shards in + * separate threads. Read records are stored in a blocking queue of limited capacity. + */ +class ShardReadersPool { + + private static final Logger LOG = LoggerFactory.getLogger(ShardReadersPool.class); + private static final int DEFAULT_CAPACITY_PER_SHARD = 10_000; + private static final int ATTEMPTS_TO_SHUTDOWN = 3; + + /** + * Executor service for running the threads that read records from shards handled by this pool. + * Each thread runs the {@link ShardReadersPool#readLoop(ShardRecordsIterator)} method and handles + * exactly one shard. + */ + private final ExecutorService executorService; + + /** + * A Bounded buffer for read records. Records are added to this buffer within {@link + * ShardReadersPool#readLoop(ShardRecordsIterator)} method and removed in {@link + * ShardReadersPool#nextRecord()}. + */ + private BlockingQueue recordsQueue; + + /** + * A reference to an immutable mapping of {@link ShardRecordsIterator} instances to shard ids. + * This map is replaced with a new one when resharding operation on any handled shard occurs. + */ + private final AtomicReference> shardIteratorsMap; + + /** A map for keeping the current number of records stored in a buffer per shard. */ + private final ConcurrentMap numberOfRecordsInAQueueByShard; + + private final SimplifiedKinesisClient kinesis; + private final WatermarkPolicyFactory watermarkPolicyFactory; + private final KinesisReaderCheckpoint initialCheckpoint; + private final int queueCapacityPerShard; + private final AtomicBoolean poolOpened = new AtomicBoolean(true); + + ShardReadersPool( + SimplifiedKinesisClient kinesis, + KinesisReaderCheckpoint initialCheckpoint, + WatermarkPolicyFactory watermarkPolicyFactory) { + this(kinesis, initialCheckpoint, watermarkPolicyFactory, DEFAULT_CAPACITY_PER_SHARD); + } + + ShardReadersPool( + SimplifiedKinesisClient kinesis, + KinesisReaderCheckpoint initialCheckpoint, + WatermarkPolicyFactory watermarkPolicyFactory, + int queueCapacityPerShard) { + this.kinesis = kinesis; + this.initialCheckpoint = initialCheckpoint; + this.watermarkPolicyFactory = watermarkPolicyFactory; + this.queueCapacityPerShard = queueCapacityPerShard; + this.executorService = Executors.newCachedThreadPool(); + this.numberOfRecordsInAQueueByShard = new ConcurrentHashMap<>(); + this.shardIteratorsMap = new AtomicReference<>(); + } + + void start() throws TransientKinesisException { + ImmutableMap.Builder shardsMap = ImmutableMap.builder(); + for (ShardCheckpoint checkpoint : initialCheckpoint) { + shardsMap.put(checkpoint.getShardId(), createShardIterator(kinesis, checkpoint)); + } + shardIteratorsMap.set(shardsMap.build()); + if (!shardIteratorsMap.get().isEmpty()) { + recordsQueue = + new ArrayBlockingQueue<>(queueCapacityPerShard * shardIteratorsMap.get().size()); + startReadingShards(shardIteratorsMap.get().values()); + } else { + // There are no shards to handle when restoring from an empty checkpoint. Empty checkpoints + // are generated when the last shard handled by this pool was closed + recordsQueue = new ArrayBlockingQueue<>(1); + } + } + + // Note: readLoop() will log any Throwable raised so opt to ignore the future result + @SuppressWarnings("FutureReturnValueIgnored") + void startReadingShards(Iterable shardRecordsIterators) { + for (final ShardRecordsIterator recordsIterator : shardRecordsIterators) { + numberOfRecordsInAQueueByShard.put(recordsIterator.getShardId(), new AtomicInteger()); + if (recordsIterator.hasConsumer()) { + LOG.info("Subscribing to shard {}", recordsIterator.getShardId()); + executorService.submit(() -> subscribeLoop(recordsIterator)); + } else { + LOG.info("Reading shard {}", recordsIterator.getShardId()); + executorService.submit(() -> readLoop(recordsIterator)); + } + } + } + + private void subscribeLoop(ShardRecordsIterator shardRecordsIterator) { + while (poolOpened.get()) { + try { + shardRecordsIterator.subscribeToShard(this::putRecord); + } catch (TransientKinesisException e) { + LOG.warn("Transient exception occurred.", e); + } catch (Throwable e) { + LOG.error("Unexpected exception occurred", e); + } + } + LOG.info("Kinesis Shard subscribe loop has finished"); + } + + private void putRecord(KinesisRecord kinesisRecord) { + try { + LOG.debug("Received record: {}", kinesisRecord); + recordsQueue.put(kinesisRecord); + numberOfRecordsInAQueueByShard.get(kinesisRecord.getShardId()).incrementAndGet(); + } catch (InterruptedException e) { + LOG.warn("Thread was interrupted, finishing the read loop", e); + Thread.currentThread().interrupt(); + poolOpened.set(false); + } + } + + private void readLoop(ShardRecordsIterator shardRecordsIterator) { + while (poolOpened.get()) { + try { + List kinesisRecords; + try { + kinesisRecords = shardRecordsIterator.readNextBatch(); + } catch (KinesisShardClosedException e) { + LOG.info( + "Shard iterator for {} shard is closed, finishing the read loop", + shardRecordsIterator.getShardId(), + e); + // Wait until all records from already closed shard are taken from the buffer and only + // then start reading successive shards. This guarantees that checkpoints will contain + // either parent or child shard and never both. Such approach allows for more + // straightforward checkpoint restoration than in a case when new shards are read + // immediately. + waitUntilAllShardRecordsRead(shardRecordsIterator); + readFromSuccessiveShards(shardRecordsIterator); + break; + } + kinesisRecords.forEach(this::putRecord); + } catch (TransientKinesisException e) { + LOG.warn("Transient exception occurred.", e); + } catch (Throwable e) { + LOG.error("Unexpected exception occurred", e); + } + } + LOG.info("Kinesis Shard read loop has finished"); + } + + CustomOptional nextRecord() { + try { + KinesisRecord record = recordsQueue.poll(1, TimeUnit.SECONDS); + if (record == null) { + return CustomOptional.absent(); + } + shardIteratorsMap.get().get(record.getShardId()).ackRecord(record); + + // numberOfRecordsInAQueueByShard contains the counter for a given shard until the shard is + // closed and then it's counter reaches 0. Thus the access here is safe + numberOfRecordsInAQueueByShard.get(record.getShardId()).decrementAndGet(); + return CustomOptional.of(record); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for KinesisRecord from the buffer"); + return CustomOptional.absent(); + } + } + + void stop() { + LOG.info("Closing shard iterators pool"); + poolOpened.set(false); + executorService.shutdown(); + awaitTermination(); + if (!executorService.isTerminated()) { + LOG.warn( + "Executor service was not completely terminated after {} attempts, trying to forcibly stop it.", + ATTEMPTS_TO_SHUTDOWN); + executorService.shutdownNow(); + awaitTermination(); + } + } + + private void awaitTermination() { + int attemptsLeft = ATTEMPTS_TO_SHUTDOWN; + boolean isTerminated = executorService.isTerminated(); + + while (!isTerminated && attemptsLeft-- > 0) { + try { + isTerminated = executorService.awaitTermination(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.error("Interrupted while waiting for the executor service to shutdown"); + throw new RuntimeException(e); + } + if (!isTerminated && attemptsLeft > 0) { + LOG.warn( + "Executor service is taking long time to shutdown, will retry. {} attempts left", + attemptsLeft); + } + } + } + + Instant getWatermark() { + return shardIteratorsMap.get().values().stream() + .map(ShardRecordsIterator::getShardWatermark) + .min(Comparator.naturalOrder()) + .orElse(BoundedWindow.TIMESTAMP_MAX_VALUE); + } + + KinesisReaderCheckpoint getCheckpointMark() { + ImmutableMap currentShardIterators = shardIteratorsMap.get(); + return new KinesisReaderCheckpoint( + currentShardIterators.values().stream() + .map( + shardRecordsIterator -> { + checkArgument( + shardRecordsIterator != null, "shardRecordsIterator can not be null"); + return shardRecordsIterator.getCheckpoint(); + }) + .collect(Collectors.toList())); + } + + ShardRecordsIterator createShardIterator( + SimplifiedKinesisClient kinesis, ShardCheckpoint checkpoint) + throws TransientKinesisException { + return new ShardRecordsIterator(checkpoint, kinesis, watermarkPolicyFactory); + } + + /** + * Waits until all records read from given shardRecordsIterator are taken from {@link + * #recordsQueue} and acked. Uses {@link #numberOfRecordsInAQueueByShard} map to track the amount + * of remaining events. + */ + private void waitUntilAllShardRecordsRead(ShardRecordsIterator shardRecordsIterator) + throws InterruptedException { + // Given shard is already closed so no more records will be read from it. Thus the counter for + // that shard will be strictly decreasing to 0. + AtomicInteger numberOfShardRecordsInAQueue = + numberOfRecordsInAQueueByShard.get(shardRecordsIterator.getShardId()); + while (!(numberOfShardRecordsInAQueue.get() == 0)) { + Thread.sleep(TimeUnit.SECONDS.toMillis(1)); + } + } + + /** + * Tries to find successors of a given shard and start reading them. Each closed shard can have 0, + * 1 or 2 successors + * + *
    + *
  • 0 successors - when shard was merged with another shard and this one is considered + * adjacent by merge operation + *
  • 1 successor - when shard was merged with another shard and this one is considered a + * parent by merge operation + *
  • 2 successors - when shard was split into two shards + *
+ * + *

Once shard successors are established, the transition to reading new shards can begin. + * During this operation, the immutable {@link ShardReadersPool#shardIteratorsMap} is replaced + * with a new one holding references to {@link ShardRecordsIterator} instances for open shards + * only. Potentially there might be more shard iterators closing at the same time so {@link + * ShardReadersPool#shardIteratorsMap} is updated in a loop using CAS pattern to keep all the + * updates. Then, the counter for already closed shard is removed from {@link + * ShardReadersPool#numberOfRecordsInAQueueByShard} map. + * + *

Finally when update is finished, new threads are spawned for reading the successive shards. + * The thread that handled reading from already closed shard can finally complete. + */ + private void readFromSuccessiveShards(final ShardRecordsIterator closedShardIterator) + throws TransientKinesisException { + List successiveShardRecordIterators = + closedShardIterator.findSuccessiveShardRecordIterators(); + + ImmutableMap current; + ImmutableMap updated; + do { + current = shardIteratorsMap.get(); + updated = + createMapWithSuccessiveShards( + current, closedShardIterator, successiveShardRecordIterators); + } while (!shardIteratorsMap.compareAndSet(current, updated)); + numberOfRecordsInAQueueByShard.remove(closedShardIterator.getShardId()); + startReadingShards(successiveShardRecordIterators); + } + + private ImmutableMap createMapWithSuccessiveShards( + ImmutableMap current, + ShardRecordsIterator closedShardIterator, + List successiveShardRecordIterators) + throws TransientKinesisException { + ImmutableMap.Builder shardsMap = ImmutableMap.builder(); + Iterable allShards = + Iterables.concat(current.values(), successiveShardRecordIterators); + for (ShardRecordsIterator iterator : allShards) { + if (!closedShardIterator.getShardId().equals(iterator.getShardId())) { + shardsMap.put(iterator.getShardId(), iterator); + } + } + return shardsMap.build(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIterator.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIterator.java new file mode 100644 index 000000000000..24ea3e11a1c8 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIterator.java @@ -0,0 +1,177 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** + * Iterates over records in a single shard. Records are retrieved in batches via calls to {@link + * ShardRecordsIterator#readNextBatch()}. Client has to confirm processed records by calling {@link + * ShardRecordsIterator#ackRecord(KinesisRecord)} method. + */ +class ShardRecordsIterator { + + private static final Logger LOG = LoggerFactory.getLogger(ShardRecordsIterator.class); + + private final SimplifiedKinesisClient kinesis; + private final RecordFilter filter; + private final String streamName; + private final String shardId; + private final Optional consumerArn; + private AtomicReference checkpoint; + private boolean resubscribe; + private String shardIterator; + private AtomicLong millisBehindLatest = new AtomicLong(Long.MAX_VALUE); + private AtomicReference watermarkPolicy; + private WatermarkPolicyFactory watermarkPolicyFactory; + + ShardRecordsIterator( + final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient, + WatermarkPolicyFactory watermarkPolicyFactory) + throws TransientKinesisException { + this(initialCheckpoint, simplifiedKinesisClient, watermarkPolicyFactory, new RecordFilter()); + } + + ShardRecordsIterator( + final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient, + WatermarkPolicyFactory watermarkPolicyFactory, + RecordFilter filter) + throws TransientKinesisException { + this.checkpoint = new AtomicReference<>(checkNotNull(initialCheckpoint, "initialCheckpoint")); + this.filter = checkNotNull(filter, "filter"); + this.kinesis = checkNotNull(simplifiedKinesisClient, "simplifiedKinesisClient"); + this.streamName = initialCheckpoint.getStreamName(); + this.shardId = initialCheckpoint.getShardId(); + this.consumerArn = Optional.ofNullable(initialCheckpoint.getConsumerArn()); + this.shardIterator = initialCheckpoint.getShardIterator(kinesis); + this.watermarkPolicy = new AtomicReference<>(watermarkPolicyFactory.createWatermarkPolicy()); + this.watermarkPolicyFactory = watermarkPolicyFactory; + } + + boolean hasConsumer() { + return consumerArn.isPresent(); + } + + void subscribeToShard(Consumer consumer) throws TransientKinesisException { + SubscribeToShardResponseHandler.Visitor visitor = + new SubscribeToShardResponseHandler.Visitor() { + @Override + public void visit(SubscribeToShardEvent event) { + LOG.debug("Received subscribe to shard event " + event); + millisBehindLatest.set(event.millisBehindLatest()); + if (!event.records().isEmpty()) { + List kinesisRecords = + SimplifiedKinesisClient.deaggregate(event.records()).stream() + .map(r -> new KinesisRecord(r, streamName, shardId)) + .collect(Collectors.toList()); + filter.apply(kinesisRecords, checkpoint.get()).forEach(consumer); + } + } + }; + checkpoint + .get() + .subscribeToShard( + resubscribe, + kinesis, + visitor, + e -> LOG.error("Error during stream - " + e.getMessage())) + .join(); + } + + List readNextBatch() + throws TransientKinesisException, KinesisShardClosedException { + if (shardIterator == null) { + throw new KinesisShardClosedException( + String.format( + "Shard iterator reached end of the shard: streamName=%s, shardId=%s", + streamName, shardId)); + } + GetKinesisRecordsResult response = fetchRecords(); + LOG.debug("Fetched {} new records", response.getRecords().size()); + + List filteredRecords = filter.apply(response.getRecords(), checkpoint.get()); + millisBehindLatest.set(response.getMillisBehindLatest()); + return filteredRecords; + } + + private GetKinesisRecordsResult fetchRecords() throws TransientKinesisException { + try { + GetKinesisRecordsResult response = kinesis.getRecords(shardIterator, streamName, shardId); + shardIterator = response.getNextShardIterator(); + return response; + } catch (ExpiredIteratorException e) { + LOG.info("Refreshing expired iterator", e); + shardIterator = checkpoint.get().getShardIterator(kinesis); + return fetchRecords(); + } + } + + ShardCheckpoint getCheckpoint() { + return checkpoint.get(); + } + + void ackRecord(KinesisRecord record) { + checkpoint.set(checkpoint.get().moveAfter(record)); + watermarkPolicy.get().update(record); + resubscribe = true; + } + + Instant getShardWatermark() { + return watermarkPolicy.get().getWatermark(); + } + + String getShardId() { + return shardId; + } + + List findSuccessiveShardRecordIterators() throws TransientKinesisException { + List shards = kinesis.listShards(streamName); + List successiveShardRecordIterators = new ArrayList<>(); + for (Shard shard : shards) { + if (shardId.equals(shard.parentShardId())) { + ShardCheckpoint shardCheckpoint = + new ShardCheckpoint( + streamName, + shard.shardId(), + consumerArn.orElse(null), + new StartingPoint(InitialPositionInStream.TRIM_HORIZON)); + successiveShardRecordIterators.add( + new ShardRecordsIterator(shardCheckpoint, kinesis, watermarkPolicyFactory)); + } + } + return successiveShardRecordIterators; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java new file mode 100644 index 000000000000..a6adc09e9e57 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java @@ -0,0 +1,283 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.joda.time.Instant; +import org.joda.time.Minutes; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.exception.SdkServiceException; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.cloudwatch.model.Datapoint; +import software.amazon.awssdk.services.cloudwatch.model.Dimension; +import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsRequest; +import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsResponse; +import software.amazon.awssdk.services.cloudwatch.model.Statistic; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.LimitExceededException; +import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.services.kinesis.model.StreamDescription; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; +import software.amazon.kinesis.retrieval.AggregatorUtil; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +/** Wraps {@link KinesisClient} class providing much simpler interface and proper error handling. */ +class SimplifiedKinesisClient { + + private static final String KINESIS_NAMESPACE = "AWS/Kinesis"; + private static final String INCOMING_RECORDS_METRIC = "IncomingBytes"; + private static final int PERIOD_GRANULARITY_IN_SECONDS = 60; + private static final String STREAM_NAME_DIMENSION = "StreamName"; + + private final KinesisClient kinesis; + private final KinesisAsyncClient kinesisAsync; + private final CloudWatchClient cloudWatch; + private final Integer limit; + + public SimplifiedKinesisClient( + KinesisClient kinesis, + KinesisAsyncClient kinesisAsync, + CloudWatchClient cloudWatch, + Integer limit) { + this.kinesis = checkNotNull(kinesis, "kinesis"); + this.kinesisAsync = checkNotNull(kinesisAsync, "kinesisAsync"); + this.cloudWatch = checkNotNull(cloudWatch, "cloudWatch"); + this.limit = limit; + } + + public static SimplifiedKinesisClient from(AWSClientsProvider provider, Integer limit) { + return new SimplifiedKinesisClient( + provider.getKinesisClient(), + provider.getKinesisAsyncClient(), + provider.getCloudWatchClient(), + limit); + } + + public CompletableFuture subscribeToShard( + final String consumerArn, + final String shardId, + final ShardIteratorType shardIteratorType, + final String startingSequenceNumber, + final Instant timestamp, + final SubscribeToShardResponseHandler.Visitor visitor, + final Consumer onError) + throws TransientKinesisException { + SubscribeToShardRequest request = + SubscribeToShardRequest.builder() + .consumerARN(consumerArn) + .shardId(shardId) + .startingPosition( + s -> + s.type(shardIteratorType) + .sequenceNumber(startingSequenceNumber) + .timestamp(TimeUtil.toJava(timestamp))) + .build(); + SubscribeToShardResponseHandler responseHandler = + SubscribeToShardResponseHandler.builder().subscriber(visitor).onError(onError).build(); + return wrapExceptions(() -> kinesisAsync.subscribeToShard(request, responseHandler)); + } + + public String getShardIterator( + final String streamName, + final String shardId, + final ShardIteratorType shardIteratorType, + final String startingSequenceNumber, + final Instant timestamp) + throws TransientKinesisException { + return wrapExceptions( + () -> + kinesis + .getShardIterator( + GetShardIteratorRequest.builder() + .streamName(streamName) + .shardId(shardId) + .shardIteratorType(shardIteratorType) + .startingSequenceNumber(startingSequenceNumber) + .timestamp(TimeUtil.toJava(timestamp)) + .build()) + .shardIterator()); + } + + public List listShards(final String streamName) throws TransientKinesisException { + return wrapExceptions( + () -> { + List shards = Lists.newArrayList(); + String lastShardId = null; + + StreamDescription description; + do { + description = + kinesis + .describeStream( + DescribeStreamRequest.builder() + .streamName(streamName) + .exclusiveStartShardId(lastShardId) + .build()) + .streamDescription(); + + shards.addAll(description.shards()); + lastShardId = shards.get(shards.size() - 1).shardId(); + } while (description.hasMoreShards()); + + return shards; + }); + } + + /** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords(String shardIterator, String streamName, String shardId) + throws TransientKinesisException { + return getRecords(shardIterator, streamName, shardId, limit); + } + + /** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords( + final String shardIterator, + final String streamName, + final String shardId, + final Integer limit) + throws TransientKinesisException { + return wrapExceptions( + () -> { + GetRecordsResponse response = + kinesis.getRecords( + GetRecordsRequest.builder().shardIterator(shardIterator).limit(limit).build()); + List records = response.records(); + return new GetKinesisRecordsResult( + deaggregate(records), + response.nextShardIterator(), + response.millisBehindLatest(), + streamName, + shardId); + }); + } + + public static List deaggregate(List records) { + return records.isEmpty() + ? ImmutableList.of() + : new AggregatorUtil() + .deaggregate( + records.stream().map(KinesisClientRecord::fromRecord).collect(Collectors.toList())); + } + + /** + * Gets total size in bytes of all events that remain in Kinesis stream after specified instant. + * + * @return total size in bytes of all Kinesis events after specified instant + */ + public long getBacklogBytes(String streamName, Instant countSince) + throws TransientKinesisException { + return getBacklogBytes(streamName, countSince, new Instant()); + } + + /** + * Gets total size in bytes of all events that remain in Kinesis stream between specified + * instants. + * + * @return total size in bytes of all Kinesis events after specified instant + */ + public long getBacklogBytes( + final String streamName, final Instant countSince, final Instant countTo) + throws TransientKinesisException { + return wrapExceptions( + () -> { + Minutes period = Minutes.minutesBetween(countSince, countTo); + if (period.isLessThan(Minutes.ONE)) { + return 0L; + } + + GetMetricStatisticsRequest request = + createMetricStatisticsRequest(streamName, countSince, countTo, period); + + long totalSizeInBytes = 0; + GetMetricStatisticsResponse response = cloudWatch.getMetricStatistics(request); + for (Datapoint point : response.datapoints()) { + totalSizeInBytes += point.sum().longValue(); + } + return totalSizeInBytes; + }); + } + + GetMetricStatisticsRequest createMetricStatisticsRequest( + String streamName, Instant countSince, Instant countTo, Minutes period) { + return GetMetricStatisticsRequest.builder() + .namespace(KINESIS_NAMESPACE) + .metricName(INCOMING_RECORDS_METRIC) + .period(period.getMinutes() * PERIOD_GRANULARITY_IN_SECONDS) + .startTime(TimeUtil.toJava(countSince)) + .endTime(TimeUtil.toJava(countTo)) + .statistics(Statistic.SUM) + .dimensions(Dimension.builder().name(STREAM_NAME_DIMENSION).value(streamName).build()) + .build(); + } + + /** + * Wraps Amazon specific exceptions into more friendly format. + * + * @throws TransientKinesisException - in case of recoverable situation, i.e. the request rate is + * too high, Kinesis remote service failed, network issue, etc. + * @throws ExpiredIteratorException - if iterator needs to be refreshed + * @throws RuntimeException - in all other cases + */ + private T wrapExceptions(Callable callable) throws TransientKinesisException { + try { + return callable.call(); + } catch (ExpiredIteratorException e) { + throw e; + } catch (LimitExceededException | ProvisionedThroughputExceededException e) { + throw new TransientKinesisException( + "Too many requests to Kinesis. Wait some time and retry.", e); + } catch (SdkServiceException e) { + throw new TransientKinesisException("Kinesis backend failed. Wait some time and retry.", e); + } catch (SdkClientException e) { + if (e.retryable()) { + throw new TransientKinesisException("Retryable client failure", e); + } + throw new RuntimeException("Not retryable client failure", e); + } catch (Exception e) { + throw new RuntimeException("Unknown kinesis failure, when trying to reach kinesis", e); + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java new file mode 100644 index 000000000000..dfeba8a7feca --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java @@ -0,0 +1,85 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Objects; +import org.joda.time.Instant; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** + * Denotes a point at which the reader should start reading from a Kinesis stream. It can be + * expressed either as an {@link InitialPositionInStream} enum constant or a timestamp, in which + * case the reader will start reading at the specified point in time. + */ +class StartingPoint implements Serializable { + + private final InitialPositionInStream position; + private final Instant timestamp; + + public StartingPoint(InitialPositionInStream position) { + this.position = checkNotNull(position, "position"); + this.timestamp = null; + } + + public StartingPoint(Instant timestamp) { + this.timestamp = checkNotNull(timestamp, "timestamp"); + this.position = null; + } + + public InitialPositionInStream getPosition() { + return position; + } + + public String getPositionName() { + return position != null ? position.name() : ShardIteratorType.AT_TIMESTAMP.name(); + } + + public Instant getTimestamp() { + return timestamp != null ? timestamp : null; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StartingPoint that = (StartingPoint) o; + return position == that.position && Objects.equals(timestamp, that.timestamp); + } + + @Override + public int hashCode() { + return Objects.hash(position, timestamp); + } + + @Override + public String toString() { + if (timestamp == null) { + return position.toString(); + } else { + return "Starting at timestamp " + timestamp; + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java new file mode 100644 index 000000000000..af74cb35fb97 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java @@ -0,0 +1,198 @@ +/* + * 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 java.io.Serializable; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; + +/** + * This class is responsible for establishing the initial set of shards that existed at the given + * starting point. + */ +class StartingPointShardsFinder implements Serializable { + + private static final Logger LOGGER = LoggerFactory.getLogger(StartingPointShardsFinder.class); + + /** + * Finds all the shards at the given startingPoint. This method starts by gathering the oldest + * shards in the stream and considers them as initial shards set. Then it validates the shards by + * getting an iterator at the given starting point and trying to read some records. If shard + * passes the validation then it is added to the result shards set. If not then it is regarded as + * expired and its successors are taken into consideration. This step is repeated until all valid + * shards are found. + * + *

The following diagram depicts sample split and merge operations on a stream with 3 initial + * shards. Let's consider what happens when T1, T2, T3 or T4 timestamps are passed as the + * startingPoint. + * + *

    + *
  • T1 timestamp (or TRIM_HORIZON marker) - 0000, 0001 and 0002 shards are the oldest so they + * are gathered as initial shards set. All of them are valid at T1 timestamp so they are all + * returned from the method. + *
  • T2 timestamp - 0000, 0001 and 0002 shards form the initial shards set. + *
      + *
    • 0000 passes the validation at T2 timestamp so it is added to the result set + *
    • 0001 does not pass the validation as it is already closed at T2 timestamp so its + * successors 0003 and 0004 are considered. Both are valid at T2 timestamp so they are + * added to the resulting set. + *
    • 0002 also does not pass the validation so its successors 0005 and 0006 are + * considered and both are valid. + *
    + * Finally the resulting set contains 0000, 0003, 0004, 0005 and 0006 shards. + *
  • T3 timestamp - the beginning is the same as in T2 case. + *
      + *
    • 0000 is valid + *
    • 0001 is already closed at T2 timestamp so its successors 0003 and 0004 are next. + * 0003 is valid but 0004 is already closed at T3 timestamp. It has one successor 0007 + * which is the result of merging 0004 and 0005 shards. 0007 has two parent shards + * then stored in {@link Shard#parentShardId()} and {@link + * Shard#adjacentParentShardId()} fields. Only one of them should follow the relation + * to its successor so it is always the shard stored in parentShardId field. Let's + * assume that it was 0004 shard and it's the one that considers 0007 its successor. + * 0007 is valid at T3 timestamp and it's added to the result set. + *
    • 0002 is closed at T3 timestamp so its successors 0005 and 0006 are next. 0005 is + * also closed because it was merged with 0004 shard. Their successor is 0007 and it + * was already considered by 0004 shard so no action here is needed. Shard 0006 is + * valid. + *
    + *
  • T4 timestamp (or LATEST marker) - following the same reasoning as in previous cases it + * end's up with 0000, 0003, 0008 and 0010 shards. + *
+ * + *
+   *      T1                T2          T3                      T4
+   *      |                 |           |                       |
+   * 0000-----------------------------------------------------------
+   *
+   *
+   *             0003-----------------------------------------------
+   *            /
+   * 0001------+
+   *            \
+   *             0004-----------+             0008------------------
+   *                             \           /
+   *                              0007------+
+   *                             /           \
+   *                  0005------+             0009------+
+   *                 /                                   \
+   * 0002-----------+                                     0010------
+   *                 \                                   /
+   *                  0006------------------------------+
+   * 
+ */ + Set findShardsAtStartingPoint( + SimplifiedKinesisClient kinesis, String streamName, StartingPoint startingPoint) + throws TransientKinesisException { + List allShards = kinesis.listShards(streamName); + Set initialShards = findInitialShardsWithoutParents(streamName, allShards); + + Set startingPointShards = new HashSet<>(); + Set expiredShards; + do { + Set validShards = validateShards(kinesis, initialShards, streamName, startingPoint); + startingPointShards.addAll(validShards); + expiredShards = Sets.difference(initialShards, validShards); + if (!expiredShards.isEmpty()) { + LOGGER.info( + "Following shards expired for {} stream at '{}' starting point: {}", + streamName, + startingPoint, + expiredShards); + } + initialShards = findNextShards(allShards, expiredShards); + } while (!expiredShards.isEmpty()); + return startingPointShards; + } + + private Set findNextShards(List allShards, Set expiredShards) { + Set nextShards = new HashSet<>(); + for (Shard expiredShard : expiredShards) { + boolean successorFound = false; + for (Shard shard : allShards) { + if (Objects.equals(expiredShard.shardId(), shard.parentShardId())) { + nextShards.add(shard); + successorFound = true; + } else if (Objects.equals(expiredShard.shardId(), shard.adjacentParentShardId())) { + successorFound = true; + } + } + if (!successorFound) { + // This can potentially happen during split/merge operation. Newly created shards might be + // not listed in the allShards list and their predecessor is already considered expired. + // Retrying should solve the issue. + throw new IllegalStateException("No successors were found for shard: " + expiredShard); + } + } + return nextShards; + } + + /** + * Finds the initial set of shards (the oldest ones). These shards do not have their parents in + * the shard list. + */ + private Set findInitialShardsWithoutParents(String streamName, List allShards) { + Set shardIds = new HashSet<>(); + for (Shard shard : allShards) { + shardIds.add(shard.shardId()); + } + LOGGER.info("Stream {} has following shards: {}", streamName, shardIds); + Set shardsWithoutParents = new HashSet<>(); + for (Shard shard : allShards) { + if (!shardIds.contains(shard.parentShardId())) { + shardsWithoutParents.add(shard); + } + } + return shardsWithoutParents; + } + + /** + * Validates the shards at the given startingPoint. Validity is checked by getting an iterator at + * the startingPoint and then trying to read some records. This action does not affect the records + * at all. If the shard is valid then it will get read from exactly the same point and these + * records will be read again. + */ + private Set validateShards( + SimplifiedKinesisClient kinesis, + Iterable rootShards, + String streamName, + StartingPoint startingPoint) + throws TransientKinesisException { + Set validShards = new HashSet<>(); + ShardIteratorType shardIteratorType = + ShardIteratorType.fromValue(startingPoint.getPositionName()); + for (Shard shard : rootShards) { + String shardIterator = + kinesis.getShardIterator( + streamName, shard.shardId(), shardIteratorType, null, startingPoint.getTimestamp()); + GetKinesisRecordsResult records = + kinesis.getRecords(shardIterator, streamName, shard.shardId()); + if (records.getNextShardIterator() != null || !records.getRecords().isEmpty()) { + validShards.add(shard); + } + } + return validShards; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StaticCheckpointGenerator.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StaticCheckpointGenerator.java new file mode 100644 index 000000000000..f8e4bfead503 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StaticCheckpointGenerator.java @@ -0,0 +1,41 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull; + +/** Always returns the same instance of checkpoint. */ +class StaticCheckpointGenerator implements CheckpointGenerator { + + private final KinesisReaderCheckpoint checkpoint; + + public StaticCheckpointGenerator(KinesisReaderCheckpoint checkpoint) { + checkNotNull(checkpoint, "checkpoint"); + this.checkpoint = checkpoint; + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) { + return checkpoint; + } + + @Override + public String toString() { + return checkpoint.toString(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java new file mode 100644 index 000000000000..95026c6b24f6 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java @@ -0,0 +1,37 @@ +/* + * 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 java.util.Optional; +import org.joda.time.Instant; + +/** Time conversion utilities. */ +public final class TimeUtil { + + public static java.time.Instant toJava(Instant timestamp) { + return Optional.ofNullable(timestamp) + .map(t -> java.time.Instant.ofEpochMilli(t.getMillis())) + .orElse(null); + } + + public static Instant toJoda(java.time.Instant instant) { + return Optional.ofNullable(instant).map(t -> new Instant(t.toEpochMilli())).orElse(null); + } + + private TimeUtil() {} +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TransientKinesisException.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TransientKinesisException.java new file mode 100644 index 000000000000..88ce40b58f1d --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TransientKinesisException.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** A transient exception thrown by Kinesis. */ +class TransientKinesisException extends Exception { + + public TransientKinesisException(String s, Throwable e) { + super(s, e); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkParameters.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkParameters.java new file mode 100644 index 000000000000..520fd3747030 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkParameters.java @@ -0,0 +1,98 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** {@code WatermarkParameters} contains the parameters used for watermark computation. */ +@AutoValue +public abstract class WatermarkParameters implements Serializable { + + private static final SerializableFunction ARRIVAL_TIME_FN = + KinesisRecord::getApproximateArrivalTimestamp; + private static final Duration STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD = + Duration.standardMinutes(2); + + abstract Instant getCurrentWatermark(); + + abstract Instant getEventTime(); + + abstract Instant getLastUpdateTime(); + + abstract SerializableFunction getTimestampFn(); + + abstract Duration getWatermarkIdleDurationThreshold(); + + public abstract Builder toBuilder(); + + public static Builder builder() { + return new AutoValue_WatermarkParameters.Builder() + .setCurrentWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setEventTime(BoundedWindow.TIMESTAMP_MIN_VALUE) + .setTimestampFn(ARRIVAL_TIME_FN) + .setLastUpdateTime(Instant.now()) + .setWatermarkIdleDurationThreshold(STANDARD_WATERMARK_IDLE_DURATION_THRESHOLD); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setCurrentWatermark(Instant currentWatermark); + + abstract Builder setEventTime(Instant eventTime); + + abstract Builder setLastUpdateTime(Instant now); + + abstract Builder setWatermarkIdleDurationThreshold(Duration watermarkIdleDurationThreshold); + + abstract Builder setTimestampFn(SerializableFunction timestampFn); + + abstract WatermarkParameters build(); + } + + public static WatermarkParameters create() { + return builder().build(); + } + + /** + * Specify the {@code SerializableFunction} to extract the event time from a {@code + * KinesisRecord}. The default event timestamp is the arrival timestamp of the record. + * + * @param timestampFn Serializable function to extract the timestamp from a record. + */ + public WatermarkParameters withTimestampFn( + SerializableFunction timestampFn) { + checkArgument(timestampFn != null, "timestampFn function is null"); + return builder().setTimestampFn(timestampFn).build(); + } + + /** + * Specify the watermark idle duration to consider before advancing the watermark. The default + * watermark idle duration threshold is 2 minutes. + */ + public WatermarkParameters withWatermarkIdleDurationThreshold(Duration idleDurationThreshold) { + checkArgument(idleDurationThreshold != null, "watermark idle duration threshold is null"); + return builder().setWatermarkIdleDurationThreshold(idleDurationThreshold).build(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicy.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicy.java new file mode 100644 index 000000000000..f1c2b242caaf --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicy.java @@ -0,0 +1,29 @@ +/* + * 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 java.io.Serializable; +import org.joda.time.Instant; + +/** Implement this interface to define a custom watermark calculation heuristic. */ +public interface WatermarkPolicy extends Serializable { + + Instant getWatermark(); + + void update(KinesisRecord record); +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyFactory.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyFactory.java new file mode 100644 index 000000000000..9d67bd516577 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyFactory.java @@ -0,0 +1,152 @@ +/* + * 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 java.io.Serializable; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Ordering; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Implement this interface to create a {@code WatermarkPolicy}. Used by the {@code + * ShardRecordsIterator} to create a watermark policy for every shard. + */ +public interface WatermarkPolicyFactory extends Serializable { + + WatermarkPolicy createWatermarkPolicy(); + + /** Returns an ArrivalTimeWatermarkPolicy. */ + static WatermarkPolicyFactory withArrivalTimePolicy() { + return ArrivalTimeWatermarkPolicy::new; + } + + /** + * Returns an ArrivalTimeWatermarkPolicy. + * + * @param watermarkIdleDurationThreshold watermark idle duration threshold. + */ + static WatermarkPolicyFactory withArrivalTimePolicy(Duration watermarkIdleDurationThreshold) { + return () -> new ArrivalTimeWatermarkPolicy(watermarkIdleDurationThreshold); + } + + /** Returns an ProcessingTimeWatermarkPolicy. */ + static WatermarkPolicyFactory withProcessingTimePolicy() { + return ProcessingTimeWatermarkPolicy::new; + } + + /** + * Returns an custom WatermarkPolicyFactory. + * + * @param watermarkParameters Watermark parameters (timestamp extractor, watermark lag) for the + * policy. + */ + static WatermarkPolicyFactory withCustomWatermarkPolicy(WatermarkParameters watermarkParameters) { + return () -> new CustomWatermarkPolicy(watermarkParameters); + } + + /** + * ArrivalTimeWatermarkPolicy uses {@link CustomWatermarkPolicy} for watermark computation. It + * uses the arrival time of the record as the event time for watermark calculations. + */ + class ArrivalTimeWatermarkPolicy implements WatermarkPolicy { + private final CustomWatermarkPolicy watermarkPolicy; + + ArrivalTimeWatermarkPolicy() { + this.watermarkPolicy = + new CustomWatermarkPolicy( + WatermarkParameters.create() + .withTimestampFn(KinesisRecord::getApproximateArrivalTimestamp)); + } + + ArrivalTimeWatermarkPolicy(Duration idleDurationThreshold) { + WatermarkParameters watermarkParameters = + WatermarkParameters.create() + .withTimestampFn(KinesisRecord::getApproximateArrivalTimestamp) + .withWatermarkIdleDurationThreshold(idleDurationThreshold); + this.watermarkPolicy = new CustomWatermarkPolicy(watermarkParameters); + } + + @Override + public Instant getWatermark() { + return watermarkPolicy.getWatermark(); + } + + @Override + public void update(KinesisRecord record) { + watermarkPolicy.update(record); + } + } + + /** + * CustomWatermarkPolicy uses parameters defined in {@link WatermarkParameters} to compute + * watermarks. This can be used as a standard heuristic to compute watermarks. Used by {@link + * ArrivalTimeWatermarkPolicy}. + */ + class CustomWatermarkPolicy implements WatermarkPolicy { + private WatermarkParameters watermarkParameters; + + CustomWatermarkPolicy(WatermarkParameters watermarkParameters) { + this.watermarkParameters = watermarkParameters; + } + + @Override + public Instant getWatermark() { + Instant now = Instant.now(); + Instant watermarkIdleThreshold = + now.minus(watermarkParameters.getWatermarkIdleDurationThreshold()); + + Instant newWatermark = + watermarkParameters.getLastUpdateTime().isBefore(watermarkIdleThreshold) + ? watermarkIdleThreshold + : watermarkParameters.getEventTime(); + + if (newWatermark.isAfter(watermarkParameters.getCurrentWatermark())) { + watermarkParameters = + watermarkParameters.toBuilder().setCurrentWatermark(newWatermark).build(); + } + return watermarkParameters.getCurrentWatermark(); + } + + @Override + public void update(KinesisRecord record) { + watermarkParameters = + watermarkParameters + .toBuilder() + .setEventTime( + Ordering.natural() + .max( + watermarkParameters.getEventTime(), + watermarkParameters.getTimestampFn().apply(record))) + .setLastUpdateTime(Instant.now()) + .build(); + } + } + + /** Watermark policy where the processing time is used as the event time. */ + class ProcessingTimeWatermarkPolicy implements WatermarkPolicy { + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public void update(KinesisRecord record) { + // do nothing + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/package-info.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/package-info.java new file mode 100644 index 000000000000..e5ab9d9f73d0 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Transforms for reading and writing from Amazon Kinesis. */ +package org.apache.beam.sdk.io.aws2.kinesis; diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java new file mode 100644 index 000000000000..c769b6f3c490 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java @@ -0,0 +1,385 @@ +/* + * 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 static java.lang.Integer.parseInt; +import static java.lang.Math.min; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.transform; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; + +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; +import org.mockito.Mockito; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisClient; +import software.amazon.awssdk.services.kinesis.model.AddTagsToStreamRequest; +import software.amazon.awssdk.services.kinesis.model.AddTagsToStreamResponse; +import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest; +import software.amazon.awssdk.services.kinesis.model.CreateStreamResponse; +import software.amazon.awssdk.services.kinesis.model.DecreaseStreamRetentionPeriodRequest; +import software.amazon.awssdk.services.kinesis.model.DecreaseStreamRetentionPeriodResponse; +import software.amazon.awssdk.services.kinesis.model.DeleteStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DeleteStreamResponse; +import software.amazon.awssdk.services.kinesis.model.DescribeLimitsRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeLimitsResponse; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerResponse; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; +import software.amazon.awssdk.services.kinesis.model.DisableEnhancedMonitoringRequest; +import software.amazon.awssdk.services.kinesis.model.DisableEnhancedMonitoringResponse; +import software.amazon.awssdk.services.kinesis.model.EnableEnhancedMonitoringRequest; +import software.amazon.awssdk.services.kinesis.model.EnableEnhancedMonitoringResponse; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse; +import software.amazon.awssdk.services.kinesis.model.IncreaseStreamRetentionPeriodRequest; +import software.amazon.awssdk.services.kinesis.model.IncreaseStreamRetentionPeriodResponse; +import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; +import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; +import software.amazon.awssdk.services.kinesis.model.ListStreamConsumersRequest; +import software.amazon.awssdk.services.kinesis.model.ListStreamConsumersResponse; +import software.amazon.awssdk.services.kinesis.model.ListStreamsRequest; +import software.amazon.awssdk.services.kinesis.model.ListStreamsResponse; +import software.amazon.awssdk.services.kinesis.model.ListTagsForStreamRequest; +import software.amazon.awssdk.services.kinesis.model.ListTagsForStreamResponse; +import software.amazon.awssdk.services.kinesis.model.MergeShardsRequest; +import software.amazon.awssdk.services.kinesis.model.MergeShardsResponse; +import software.amazon.awssdk.services.kinesis.model.PutRecordRequest; +import software.amazon.awssdk.services.kinesis.model.PutRecordResponse; +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerRequest; +import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerResponse; +import software.amazon.awssdk.services.kinesis.model.RemoveTagsFromStreamRequest; +import software.amazon.awssdk.services.kinesis.model.RemoveTagsFromStreamResponse; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.services.kinesis.model.SplitShardRequest; +import software.amazon.awssdk.services.kinesis.model.SplitShardResponse; +import software.amazon.awssdk.services.kinesis.model.StartStreamEncryptionRequest; +import software.amazon.awssdk.services.kinesis.model.StartStreamEncryptionResponse; +import software.amazon.awssdk.services.kinesis.model.StopStreamEncryptionRequest; +import software.amazon.awssdk.services.kinesis.model.StopStreamEncryptionResponse; +import software.amazon.awssdk.services.kinesis.model.UpdateShardCountRequest; +import software.amazon.awssdk.services.kinesis.model.UpdateShardCountResponse; + +/** Mock implementation of {@link KinesisClient} for testing. */ +class AmazonKinesisMock implements KinesisClient { + + static class TestData implements Serializable { + + private final String data; + private final Instant arrivalTimestamp; + private final String sequenceNumber; + + public TestData(KinesisRecord record) { + this( + new String(record.getData().array(), StandardCharsets.UTF_8), + record.getApproximateArrivalTimestamp(), + record.getSequenceNumber()); + } + + public TestData(String data, Instant arrivalTimestamp, String sequenceNumber) { + this.data = data; + this.arrivalTimestamp = arrivalTimestamp; + this.sequenceNumber = sequenceNumber; + } + + public Record convertToRecord() { + return Record.builder() + .approximateArrivalTimestamp(TimeUtil.toJava(arrivalTimestamp)) + .data(SdkBytes.fromByteArray(data.getBytes(StandardCharsets.UTF_8))) + .sequenceNumber(sequenceNumber) + .partitionKey("") + .build(); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + + @Override + public String toString() { + return "TestData{" + + "data='" + + data + + '\'' + + ", arrivalTimestamp=" + + arrivalTimestamp + + ", sequenceNumber='" + + sequenceNumber + + '\'' + + '}'; + } + } + + static class Provider implements AWSClientsProvider { + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public Provider(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public KinesisClient getKinesisClient() { + return new AmazonKinesisMock( + shardedData.stream() + .map(testDatas -> transform(testDatas, TestData::convertToRecord)) + .collect(Collectors.toList()), + numberOfRecordsPerGet); + } + + @Override + public KinesisAsyncClient getKinesisAsyncClient() { + return Mockito.mock(KinesisAsyncClient.class); + } + + @Override + public CloudWatchClient getCloudWatchClient() { + return Mockito.mock(CloudWatchClient.class); + } + } + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public AmazonKinesisMock(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public String serviceName() { + return null; + } + + @Override + public void close() {} + + @Override + public GetRecordsResponse getRecords(GetRecordsRequest getRecordsRequest) { + List shardIteratorParts = + Splitter.on(':').splitToList(getRecordsRequest.shardIterator()); + int shardId = parseInt(shardIteratorParts.get(0)); + int startingRecord = parseInt(shardIteratorParts.get(1)); + List shardData = shardedData.get(shardId); + + int toIndex = min(startingRecord + numberOfRecordsPerGet, shardData.size()); + int fromIndex = min(startingRecord, toIndex); + return GetRecordsResponse.builder() + .records(shardData.subList(fromIndex, toIndex)) + .nextShardIterator(String.format("%s:%s", shardId, toIndex)) + .millisBehindLatest(0L) + .build(); + } + + @Override + public GetShardIteratorResponse getShardIterator( + GetShardIteratorRequest getShardIteratorRequest) { + ShardIteratorType shardIteratorType = getShardIteratorRequest.shardIteratorType(); + + String shardIterator; + if (shardIteratorType == ShardIteratorType.TRIM_HORIZON) { + shardIterator = String.format("%s:%s", getShardIteratorRequest.shardId(), 0); + } else { + throw new RuntimeException("Not implemented"); + } + + return GetShardIteratorResponse.builder().shardIterator(shardIterator).build(); + } + + @Override + public DescribeStreamResponse describeStream(DescribeStreamRequest describeStreamRequest) { + int nextShardId = 0; + if (describeStreamRequest.exclusiveStartShardId() != null) { + nextShardId = parseInt(describeStreamRequest.exclusiveStartShardId()) + 1; + } + boolean hasMoreShards = nextShardId + 1 < shardedData.size(); + + List shards = new ArrayList<>(); + if (nextShardId < shardedData.size()) { + shards.add(Shard.builder().shardId(Integer.toString(nextShardId)).build()); + } + + DescribeStreamResponse.Builder builder = + DescribeStreamResponse.builder() + .streamDescription( + s -> + s.hasMoreShards(hasMoreShards) + .shards(shards) + .streamName(describeStreamRequest.streamName())); + builder.sdkHttpResponse(SdkHttpResponse.builder().statusCode(200).build()); + return builder.build(); + } + + @Override + public AddTagsToStreamResponse addTagsToStream(AddTagsToStreamRequest addTagsToStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public CreateStreamResponse createStream(CreateStreamRequest createStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DecreaseStreamRetentionPeriodResponse decreaseStreamRetentionPeriod( + DecreaseStreamRetentionPeriodRequest decreaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DeleteStreamResponse deleteStream(DeleteStreamRequest deleteStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeLimitsResponse describeLimits(DescribeLimitsRequest describeLimitsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamConsumerResponse describeStreamConsumer( + DescribeStreamConsumerRequest describeStreamConsumerRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamSummaryResponse describeStreamSummary( + DescribeStreamSummaryRequest describeStreamSummaryRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DisableEnhancedMonitoringResponse disableEnhancedMonitoring( + DisableEnhancedMonitoringRequest disableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public EnableEnhancedMonitoringResponse enableEnhancedMonitoring( + EnableEnhancedMonitoringRequest enableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public IncreaseStreamRetentionPeriodResponse increaseStreamRetentionPeriod( + IncreaseStreamRetentionPeriodRequest increaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListShardsResponse listShards(ListShardsRequest listShardsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamConsumersResponse listStreamConsumers( + ListStreamConsumersRequest listStreamConsumersRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResponse listStreams(ListStreamsRequest listStreamsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResponse listStreams() { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListTagsForStreamResponse listTagsForStream( + ListTagsForStreamRequest listTagsForStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public MergeShardsResponse mergeShards(MergeShardsRequest mergeShardsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResponse putRecord(PutRecordRequest putRecordRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordsResponse putRecords(PutRecordsRequest putRecordsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public RegisterStreamConsumerResponse registerStreamConsumer( + RegisterStreamConsumerRequest registerStreamConsumerRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public RemoveTagsFromStreamResponse removeTagsFromStream( + RemoveTagsFromStreamRequest removeTagsFromStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public SplitShardResponse splitShard(SplitShardRequest splitShardRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public StartStreamEncryptionResponse startStreamEncryption( + StartStreamEncryptionRequest startStreamEncryptionRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public StopStreamEncryptionResponse stopStreamEncryption( + StopStreamEncryptionRequest stopStreamEncryptionRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public UpdateShardCountResponse updateShardCount( + UpdateShardCountRequest updateShardCountRequest) { + throw new RuntimeException("Not implemented"); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptionalTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptionalTest.java new file mode 100644 index 000000000000..de3043371f36 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/CustomOptionalTest.java @@ -0,0 +1,44 @@ +/* + * 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.common.testing.EqualsTester; +import java.util.NoSuchElementException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests {@link CustomOptional}. */ +@RunWith(JUnit4.class) +public class CustomOptionalTest { + + @Test(expected = NoSuchElementException.class) + public void absentThrowsNoSuchElementExceptionOnGet() { + CustomOptional.absent().get(); + } + + @Test + public void testEqualsAndHashCode() { + new EqualsTester() + .addEqualityGroup(CustomOptional.absent(), CustomOptional.absent()) + .addEqualityGroup(CustomOptional.of(3), CustomOptional.of(3)) + .addEqualityGroup(CustomOptional.of(11)) + .addEqualityGroup(CustomOptional.of("3")) + .testEquals(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java new file mode 100644 index 000000000000..b36372dfc472 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java @@ -0,0 +1,80 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +import java.util.Set; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** * */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(Shard.class) +public class DynamicCheckpointGeneratorTest { + + @Mock private SimplifiedKinesisClient kinesisClient; + @Mock private StartingPointShardsFinder startingPointShardsFinder; + @Mock private Shard shard1, shard2, shard3; + + @Test + public void shouldMapAllShardsToCheckpoints() throws Exception { + when(shard1.shardId()).thenReturn("shard-01"); + when(shard2.shardId()).thenReturn("shard-02"); + when(shard3.shardId()).thenReturn("shard-03"); + Set shards = Sets.newHashSet(shard1, shard2, shard3); + StartingPoint startingPoint = new StartingPoint(InitialPositionInStream.LATEST); + when(startingPointShardsFinder.findShardsAtStartingPoint( + kinesisClient, "stream", startingPoint)) + .thenReturn(shards); + DynamicCheckpointGenerator underTest = + new DynamicCheckpointGenerator("stream", null, startingPoint, startingPointShardsFinder); + + KinesisReaderCheckpoint checkpoint = underTest.generate(kinesisClient); + + assertThat(checkpoint).hasSize(3); + } + + @Test + public void shouldMapAllValidShardsToCheckpoints() throws Exception { + when(shard1.shardId()).thenReturn("shard-01"); + when(shard2.shardId()).thenReturn("shard-02"); + when(shard3.shardId()).thenReturn("shard-03"); + String streamName = "stream"; + Set shards = Sets.newHashSet(shard1, shard2); + StartingPoint startingPoint = new StartingPoint(InitialPositionInStream.LATEST); + when(startingPointShardsFinder.findShardsAtStartingPoint( + kinesisClient, "stream", startingPoint)) + .thenReturn(shards); + + DynamicCheckpointGenerator underTest = + new DynamicCheckpointGenerator(streamName, null, startingPoint, startingPointShardsFinder); + + KinesisReaderCheckpoint checkpoint = underTest.generate(kinesisClient); + assertThat(checkpoint) + .hasSize(2) + .doesNotContain(new ShardCheckpoint(streamName, shard3.shardId(), null, startingPoint)); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java new file mode 100644 index 000000000000..565e03cc0140 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java @@ -0,0 +1,155 @@ +/* + * 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.amazonaws.regions.Regions; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.Random; +import org.apache.beam.sdk.io.GenerateSequence; +import org.apache.beam.sdk.io.common.HashingFn; +import org.apache.beam.sdk.io.common.TestRow; +import org.apache.beam.sdk.io.kinesis.KinesisPartitioner; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import software.amazon.awssdk.regions.Region; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** + * Integration test, that writes and reads data to and from real Kinesis. You need to provide {@link + * KinesisTestOptions} in order to run this. + */ +@RunWith(JUnit4.class) +public class KinesisIOIT implements Serializable { + private static int numberOfShards; + private static int numberOfRows; + + @Rule public TestPipeline pipelineWrite = TestPipeline.create(); + @Rule public TestPipeline pipelineRead = TestPipeline.create(); + + private static KinesisTestOptions options; + private static final Instant now = Instant.now(); + + @BeforeClass + public static void setup() { + PipelineOptionsFactory.register(KinesisTestOptions.class); + options = TestPipeline.testingPipelineOptions().as(KinesisTestOptions.class); + numberOfShards = options.getNumberOfShards(); + numberOfRows = options.getNumberOfRecords(); + } + + /** Test which write and then read data for a Kinesis stream. */ + @Test + public void testWriteThenRead() { + runWrite(); + runRead(); + } + + /** Write test dataset into Kinesis stream. */ + private void runWrite() { + pipelineWrite + .apply("Generate Sequence", GenerateSequence.from(0).to((long) numberOfRows)) + .apply("Prepare TestRows", ParDo.of(new TestRow.DeterministicallyConstructTestRowFn())) + .apply("Prepare Kinesis input records", ParDo.of(new ConvertToBytes())) + .apply( + "Write to Kinesis", + org.apache.beam.sdk.io.kinesis.KinesisIO.write() + .withStreamName(options.getAwsKinesisStream()) + .withPartitioner(new RandomPartitioner()) + .withAWSClientsProvider( + options.getAwsAccessKey(), + options.getAwsSecretKey(), + Regions.fromName(options.getAwsKinesisRegion()))); + + pipelineWrite.run().waitUntilFinish(); + } + + /** Read test dataset from Kinesis stream. */ + private void runRead() { + PCollection output = + pipelineRead.apply( + KinesisIO.read() + .withStreamName(options.getAwsKinesisStream()) + .withAWSClientsProvider( + options.getAwsAccessKey(), + options.getAwsSecretKey(), + Region.of(options.getAwsKinesisRegion())) + .withMaxNumRecords(numberOfRows) + // to prevent endless running in case of error + .withMaxReadTime(Duration.standardMinutes(10)) + .withInitialPositionInStream(InitialPositionInStream.AT_TIMESTAMP) + .withInitialTimestampInStream(now) + .withRequestRecordsLimit(1000)); + + PAssert.thatSingleton(output.apply("Count All", Count.globally())) + .isEqualTo((long) numberOfRows); + + PCollection consolidatedHashcode = + output + .apply(ParDo.of(new ExtractDataValues())) + .apply("Hash row contents", Combine.globally(new HashingFn()).withoutDefaults()); + + PAssert.that(consolidatedHashcode) + .containsInAnyOrder(TestRow.getExpectedHashForRowCount(numberOfRows)); + + pipelineRead.run().waitUntilFinish(); + } + + /** Produces test rows. */ + private static class ConvertToBytes extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(String.valueOf(c.element().name()).getBytes(StandardCharsets.UTF_8)); + } + } + + /** Read rows from Table. */ + private static class ExtractDataValues extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(new String(c.element().getDataAsBytes(), StandardCharsets.UTF_8)); + } + } + + private static final class RandomPartitioner implements KinesisPartitioner { + @Override + public String getPartitionKey(byte[] value) { + Random rand = new Random(); + int n = rand.nextInt(numberOfShards) + 1; + return String.valueOf(n); + } + + @Override + public String getExplicitHashKey(byte[] value) { + return null; + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisMockReadTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisMockReadTest.java new file mode 100644 index 000000000000..6a719333922b --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisMockReadTest.java @@ -0,0 +1,93 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.newArrayList; + +import java.util.List; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.joda.time.DateTime; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import software.amazon.kinesis.common.InitialPositionInStream; + +/** Tests {@link AmazonKinesisMock}. */ +@RunWith(JUnit4.class) +public class KinesisMockReadTest { + + @Rule public final transient TestPipeline p = TestPipeline.create(); + + @Test + public void readsDataFromMockKinesis() { + int noOfShards = 3; + int noOfEventsPerShard = 100; + List> testData = + provideTestData(noOfShards, noOfEventsPerShard); + + PCollection result = + p.apply( + KinesisIO.read() + .withStreamName("stream") + .withInitialPositionInStream(InitialPositionInStream.TRIM_HORIZON) + .withAWSClientsProvider(new AmazonKinesisMock.Provider(testData, 10)) + .withArrivalTimeWatermarkPolicy() + .withMaxNumRecords(noOfShards * noOfEventsPerShard)) + .apply(ParDo.of(new KinesisRecordToTestData())); + PAssert.that(result).containsInAnyOrder(Iterables.concat(testData)); + p.run(); + } + + static class KinesisRecordToTestData extends DoFn { + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + c.output(new AmazonKinesisMock.TestData(c.element())); + } + } + + private List> provideTestData( + int noOfShards, int noOfEventsPerShard) { + + int seqNumber = 0; + + List> shardedData = newArrayList(); + for (int i = 0; i < noOfShards; ++i) { + List shardData = newArrayList(); + shardedData.add(shardData); + + DateTime arrival = DateTime.now(); + for (int j = 0; j < noOfEventsPerShard; ++j) { + arrival = arrival.plusSeconds(1); + + seqNumber++; + shardData.add( + new AmazonKinesisMock.TestData( + Integer.toString(seqNumber), arrival.toInstant(), Integer.toString(seqNumber))); + } + } + + return shardedData; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java new file mode 100644 index 000000000000..aab5c1d677bb --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java @@ -0,0 +1,64 @@ +/* + * 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 static java.util.Arrays.asList; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Iterator; +import java.util.List; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +/** * */ +@RunWith(MockitoJUnitRunner.class) +public class KinesisReaderCheckpointTest { + + @Mock private ShardCheckpoint a, b, c; + + private KinesisReaderCheckpoint checkpoint; + + @Before + public void setUp() { + checkpoint = new KinesisReaderCheckpoint(asList(a, b, c)); + } + + @Test + public void splitsCheckpointAccordingly() { + verifySplitInto(1); + verifySplitInto(2); + verifySplitInto(3); + verifySplitInto(4); + } + + @Test(expected = UnsupportedOperationException.class) + public void isImmutable() { + Iterator iterator = checkpoint.iterator(); + iterator.remove(); + } + + private void verifySplitInto(int size) { + List split = checkpoint.splitInto(size); + assertThat(Iterables.concat(split)).containsOnly(a, b, c); + assertThat(split).hasSize(Math.min(size, 3)); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java new file mode 100644 index 000000000000..f4b07af12d01 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java @@ -0,0 +1,183 @@ +/* + * 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 static java.util.Arrays.asList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +/** Tests {@link KinesisReader}. */ +@RunWith(MockitoJUnitRunner.Silent.class) +public class KinesisReaderTest { + + @Mock private SimplifiedKinesisClient kinesis; + @Mock private CheckpointGenerator generator; + @Mock private ShardCheckpoint firstCheckpoint, secondCheckpoint; + @Mock private KinesisRecord a, b, c, d; + @Mock private KinesisSource kinesisSource; + @Mock private ShardReadersPool shardReadersPool; + + private KinesisReader reader; + + @Before + public void setUp() throws TransientKinesisException { + when(generator.generate(kinesis)) + .thenReturn(new KinesisReaderCheckpoint(asList(firstCheckpoint, secondCheckpoint))); + when(shardReadersPool.nextRecord()).thenReturn(CustomOptional.absent()); + when(a.getApproximateArrivalTimestamp()).thenReturn(Instant.now()); + when(b.getApproximateArrivalTimestamp()).thenReturn(Instant.now()); + when(c.getApproximateArrivalTimestamp()).thenReturn(Instant.now()); + when(d.getApproximateArrivalTimestamp()).thenReturn(Instant.now()); + + reader = spy(createReader(Duration.ZERO)); + } + + private KinesisReader createReader(Duration backlogBytesCheckThreshold) { + return new KinesisReader( + kinesis, + generator, + kinesisSource, + WatermarkPolicyFactory.withArrivalTimePolicy(), + Duration.ZERO, + backlogBytesCheckThreshold) { + @Override + ShardReadersPool createShardReadersPool() { + return shardReadersPool; + } + }; + } + + @Test + public void startReturnsFalseIfNoDataAtTheBeginning() throws IOException { + assertThat(reader.start()).isFalse(); + } + + @Test(expected = NoSuchElementException.class) + public void throwsNoSuchElementExceptionIfNoData() throws IOException { + reader.start(); + reader.getCurrent(); + } + + @Test + public void startReturnsTrueIfSomeDataAvailable() throws IOException { + when(shardReadersPool.nextRecord()) + .thenReturn(CustomOptional.of(a)) + .thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + } + + @Test + public void readsThroughAllDataAvailable() throws IOException { + when(shardReadersPool.nextRecord()) + .thenReturn(CustomOptional.of(c)) + .thenReturn(CustomOptional.absent()) + .thenReturn(CustomOptional.of(a)) + .thenReturn(CustomOptional.absent()) + .thenReturn(CustomOptional.of(d)) + .thenReturn(CustomOptional.of(b)) + .thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(c); + assertThat(reader.advance()).isFalse(); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(a); + assertThat(reader.advance()).isFalse(); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(d); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(b); + assertThat(reader.advance()).isFalse(); + } + + @Test + public void returnsCurrentWatermark() throws IOException { + Instant expectedWatermark = new Instant(123456L); + when(shardReadersPool.getWatermark()).thenReturn(expectedWatermark); + + reader.start(); + Instant currentWatermark = reader.getWatermark(); + + assertThat(currentWatermark).isEqualTo(expectedWatermark); + } + + @Test + public void getTotalBacklogBytesShouldReturnLastSeenValueWhenKinesisExceptionsOccur() + throws TransientKinesisException, IOException { + reader.start(); + when(kinesisSource.getStreamName()).thenReturn("stream1"); + doReturn(Instant.now().minus(Duration.standardMinutes(1))).when(reader).getWatermark(); + when(kinesis.getBacklogBytes(eq("stream1"), any(Instant.class))) + .thenReturn(10L) + .thenThrow(TransientKinesisException.class) + .thenReturn(20L); + + assertThat(reader.getTotalBacklogBytes()).isEqualTo(10); + assertThat(reader.getTotalBacklogBytes()).isEqualTo(10); + assertThat(reader.getTotalBacklogBytes()).isEqualTo(20); + } + + @Test + public void getTotalBacklogBytesShouldReturnLastSeenValueWhenCalledFrequently() + throws TransientKinesisException, IOException { + KinesisReader backlogCachingReader = spy(createReader(Duration.standardSeconds(30))); + backlogCachingReader.start(); + doReturn(Instant.now().minus(Duration.standardMinutes(1))) + .when(backlogCachingReader) + .getWatermark(); + when(kinesisSource.getStreamName()).thenReturn("stream1"); + when(kinesis.getBacklogBytes(eq("stream1"), any(Instant.class))) + .thenReturn(10L) + .thenReturn(20L); + + assertThat(backlogCachingReader.getTotalBacklogBytes()).isEqualTo(10); + assertThat(backlogCachingReader.getTotalBacklogBytes()).isEqualTo(10); + } + + @Test + public void getTotalBacklogBytesShouldReturnBacklogUnknown() + throws IOException, TransientKinesisException { + reader.start(); + when(kinesisSource.getStreamName()).thenReturn("stream1"); + when(reader.getWatermark()) + .thenReturn(BoundedWindow.TIMESTAMP_MIN_VALUE) + .thenReturn(Instant.now().minus(Duration.standardMinutes(1))); + when(kinesis.getBacklogBytes(eq("stream1"), any(Instant.class))).thenReturn(10L); + + assertThat(reader.getTotalBacklogBytes()) + .isEqualTo(UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN); + assertThat(reader.getTotalBacklogBytes()).isEqualTo(10); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoderTest.java new file mode 100644 index 000000000000..f7848d7acc2d --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecordCoderTest.java @@ -0,0 +1,46 @@ +/* + * 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 java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.testing.CoderProperties; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests {@link KinesisRecordCoder}. */ +@RunWith(JUnit4.class) +public class KinesisRecordCoderTest { + + @Test + public void encodingAndDecodingWorks() throws Exception { + KinesisRecord record = + new KinesisRecord( + ByteBuffer.wrap("data".getBytes(StandardCharsets.UTF_8)), + "sequence", + 128L, + "partition", + Instant.now(), + Instant.now(), + "stream", + "shard"); + CoderProperties.coderDecodeEncodeEqual(new KinesisRecordCoder(), record); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java new file mode 100644 index 000000000000..ede988aa76c1 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java @@ -0,0 +1,84 @@ +/* + * 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 static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists.newArrayList; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.joda.time.DateTime; + +/** Simple mock implementation of Kinesis service for testing, singletone. */ +public class KinesisServiceMock { + private static KinesisServiceMock instance; + + // Mock stream where client is supposed to write + private String existedStream; + + private AtomicInteger addedRecords = new AtomicInteger(0); + private AtomicInteger seqNumber = new AtomicInteger(0); + private List> shardedData; + + private KinesisServiceMock() {} + + public static synchronized KinesisServiceMock getInstance() { + if (instance == null) { + instance = new KinesisServiceMock(); + } + return instance; + } + + public synchronized void init(String stream, int shardsNum) { + existedStream = stream; + addedRecords.set(0); + seqNumber.set(0); + shardedData = newArrayList(); + for (int i = 0; i < shardsNum; i++) { + List shardData = newArrayList(); + shardedData.add(shardData); + } + } + + public AtomicInteger getAddedRecords() { + return addedRecords; + } + + public String getExistedStream() { + return existedStream; + } + + public synchronized void addShardedData(ByteBuffer data, DateTime arrival) { + String dataString = StandardCharsets.UTF_8.decode(data).toString(); + + List shardData = shardedData.get(0); + + seqNumber.incrementAndGet(); + AmazonKinesisMock.TestData testData = + new AmazonKinesisMock.TestData( + dataString, arrival.toInstant(), Integer.toString(seqNumber.get())); + shardData.add(testData); + + addedRecords.incrementAndGet(); + } + + public synchronized List> getShardedData() { + return shardedData; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTestOptions.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTestOptions.java new file mode 100644 index 000000000000..33cddc89f955 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisTestOptions.java @@ -0,0 +1,62 @@ +/* + * 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 org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.testing.TestPipelineOptions; + +/** Options for Kinesis integration tests. */ +public interface KinesisTestOptions extends TestPipelineOptions { + + @Description("AWS region where Kinesis stream resided") + @Default.String("aws-kinesis-region") + String getAwsKinesisRegion(); + + void setAwsKinesisRegion(String value); + + @Description("Kinesis stream name") + @Default.String("aws-kinesis-stream") + String getAwsKinesisStream(); + + void setAwsKinesisStream(String value); + + @Description("AWS secret key") + @Default.String("aws-secret-key") + String getAwsSecretKey(); + + void setAwsSecretKey(String value); + + @Description("AWS access key") + @Default.String("aws-access-key") + String getAwsAccessKey(); + + void setAwsAccessKey(String value); + + @Description("Number of shards of stream") + @Default.Integer(2) + Integer getNumberOfShards(); + + void setNumberOfShards(Integer count); + + @Description("Number of records that will be written and read by the test") + @Default.Integer(1000) + Integer getNumberOfRecords(); + + void setNumberOfRecords(Integer count); +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java new file mode 100644 index 000000000000..0058e2ea3166 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java @@ -0,0 +1,62 @@ +/* + * 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 static org.mockito.Mockito.when; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +/** * */ +@RunWith(MockitoJUnitRunner.class) +public class RecordFilterTest { + + @Mock private ShardCheckpoint checkpoint; + @Mock private KinesisRecord record1, record2, record3, record4, record5; + + @Test + public void shouldFilterOutRecordsBeforeOrAtCheckpoint() { + when(checkpoint.isBeforeOrAt(record1)).thenReturn(false); + when(checkpoint.isBeforeOrAt(record2)).thenReturn(true); + when(checkpoint.isBeforeOrAt(record3)).thenReturn(true); + when(checkpoint.isBeforeOrAt(record4)).thenReturn(false); + when(checkpoint.isBeforeOrAt(record5)).thenReturn(true); + List records = Lists.newArrayList(record1, record2, record3, record4, record5); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).containsOnly(record2, record3, record5); + } + + @Test + public void shouldNotFailOnEmptyList() { + List records = Collections.emptyList(); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).isEmpty(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java new file mode 100644 index 000000000000..cebd27a2b900 --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java @@ -0,0 +1,162 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static software.amazon.awssdk.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import static software.amazon.kinesis.common.InitialPositionInStream.LATEST; +import static software.amazon.kinesis.common.InitialPositionInStream.TRIM_HORIZON; + +import java.io.IOException; +import org.joda.time.DateTime; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +/** */ +@RunWith(MockitoJUnitRunner.class) +public class ShardCheckpointTest { + + private static final String AT_SEQUENCE_SHARD_IT = "AT_SEQUENCE_SHARD_IT"; + private static final String AFTER_SEQUENCE_SHARD_IT = "AFTER_SEQUENCE_SHARD_IT"; + private static final String STREAM_NAME = "STREAM"; + private static final String SHARD_ID = "SHARD_ID"; + @Mock private SimplifiedKinesisClient client; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(client.getShardIterator( + eq(STREAM_NAME), + eq(SHARD_ID), + eq(AT_SEQUENCE_NUMBER), + anyString(), + isNull(Instant.class))) + .thenReturn(AT_SEQUENCE_SHARD_IT); + when(client.getShardIterator( + eq(STREAM_NAME), + eq(SHARD_ID), + eq(AFTER_SEQUENCE_NUMBER), + anyString(), + isNull(Instant.class))) + .thenReturn(AFTER_SEQUENCE_SHARD_IT); + } + + @Test + public void testProvidingShardIterator() throws IOException, TransientKinesisException { + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AFTER_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + } + + @Test + public void testComparisonWithExtendedSequenceNumber() { + assertThat( + new ShardCheckpoint("", "", null, new StartingPoint(LATEST)) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isTrue(); + + assertThat( + new ShardCheckpoint("", "", null, new StartingPoint(TRIM_HORIZON)) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isTrue(); + + assertThat( + checkpoint(AFTER_SEQUENCE_NUMBER, "10", 1L) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isTrue(); + + assertThat( + checkpoint(AT_SEQUENCE_NUMBER, "100", 0L) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isTrue(); + + assertThat( + checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isFalse(); + + assertThat( + checkpoint(AT_SEQUENCE_NUMBER, "100", 1L) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("100", 0L)))) + .isFalse(); + + assertThat( + checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L) + .isBeforeOrAt(recordWith(new ExtendedSequenceNumber("99", 1L)))) + .isFalse(); + } + + @Test + public void testComparisonWithTimestamp() { + DateTime referenceTimestamp = DateTime.now(); + + assertThat( + checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.minusMillis(10).toInstant()))) + .isFalse(); + + assertThat( + checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.toInstant()))) + .isTrue(); + + assertThat( + checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.plusMillis(10).toInstant()))) + .isTrue(); + } + + private KinesisRecord recordWith(ExtendedSequenceNumber extendedSequenceNumber) { + KinesisRecord record = mock(KinesisRecord.class); + when(record.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); + return record; + } + + private ShardCheckpoint checkpoint( + ShardIteratorType iteratorType, String sequenceNumber, Long subSequenceNumber) { + return new ShardCheckpoint( + STREAM_NAME, SHARD_ID, null, iteratorType, sequenceNumber, subSequenceNumber); + } + + private KinesisRecord recordWith(Instant approximateArrivalTimestamp) { + KinesisRecord record = mock(KinesisRecord.class); + when(record.getApproximateArrivalTimestamp()).thenReturn(approximateArrivalTimestamp); + return record; + } + + private ShardCheckpoint checkpoint(ShardIteratorType iteratorType, Instant timestamp) { + return new ShardCheckpoint(STREAM_NAME, SHARD_ID, null, iteratorType, timestamp); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java new file mode 100644 index 000000000000..8c4300ed73de --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java @@ -0,0 +1,283 @@ +/* + * 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 static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stopwatch; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; + +/** Tests {@link ShardReadersPool}. */ +@RunWith(MockitoJUnitRunner.Silent.class) +public class ShardReadersPoolTest { + + private static final int TIMEOUT_IN_MILLIS = (int) TimeUnit.SECONDS.toMillis(10); + + @Mock private ShardRecordsIterator firstIterator, secondIterator, thirdIterator, fourthIterator; + @Mock private ShardCheckpoint firstCheckpoint, secondCheckpoint; + @Mock private SimplifiedKinesisClient kinesis; + @Mock private KinesisRecord a, b, c, d; + @Mock private WatermarkPolicyFactory factory; + + private KinesisReaderCheckpoint checkpoint; + private ShardReadersPool shardReadersPool; + private final Instant now = Instant.now(); + + @Before + public void setUp() throws TransientKinesisException { + when(a.getShardId()).thenReturn("shard1"); + when(b.getShardId()).thenReturn("shard1"); + when(c.getShardId()).thenReturn("shard2"); + when(d.getShardId()).thenReturn("shard2"); + when(firstCheckpoint.getShardId()).thenReturn("shard1"); + when(secondCheckpoint.getShardId()).thenReturn("shard2"); + when(firstIterator.getShardId()).thenReturn("shard1"); + when(firstIterator.getCheckpoint()).thenReturn(firstCheckpoint); + when(secondIterator.getShardId()).thenReturn("shard2"); + when(secondIterator.getCheckpoint()).thenReturn(secondCheckpoint); + when(thirdIterator.getShardId()).thenReturn("shard3"); + when(fourthIterator.getShardId()).thenReturn("shard4"); + + WatermarkPolicy policy = WatermarkPolicyFactory.withArrivalTimePolicy().createWatermarkPolicy(); + + checkpoint = new KinesisReaderCheckpoint(ImmutableList.of(firstCheckpoint, secondCheckpoint)); + shardReadersPool = Mockito.spy(new ShardReadersPool(kinesis, checkpoint, factory)); + + when(factory.createWatermarkPolicy()).thenReturn(policy); + + doReturn(firstIterator).when(shardReadersPool).createShardIterator(kinesis, firstCheckpoint); + doReturn(secondIterator).when(shardReadersPool).createShardIterator(kinesis, secondCheckpoint); + } + + @After + public void clean() { + shardReadersPool.stop(); + } + + @Test + public void shouldReturnAllRecords() + throws TransientKinesisException, KinesisShardClosedException { + when(firstIterator.readNextBatch()) + .thenReturn(Collections.emptyList()) + .thenReturn(ImmutableList.of(a, b)) + .thenReturn(Collections.emptyList()); + when(secondIterator.readNextBatch()) + .thenReturn(singletonList(c)) + .thenReturn(singletonList(d)) + .thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + List fetchedRecords = new ArrayList<>(); + while (fetchedRecords.size() < 4) { + CustomOptional nextRecord = shardReadersPool.nextRecord(); + if (nextRecord.isPresent()) { + fetchedRecords.add(nextRecord.get()); + } + } + assertThat(fetchedRecords).containsExactlyInAnyOrder(a, b, c, d); + } + + @Test + public void shouldReturnAbsentOptionalWhenNoRecords() + throws TransientKinesisException, KinesisShardClosedException { + when(firstIterator.readNextBatch()).thenReturn(Collections.emptyList()); + when(secondIterator.readNextBatch()).thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + CustomOptional nextRecord = shardReadersPool.nextRecord(); + assertThat(nextRecord.isPresent()).isFalse(); + } + + @Test + public void shouldCheckpointReadRecords() + throws TransientKinesisException, KinesisShardClosedException { + when(firstIterator.readNextBatch()) + .thenReturn(ImmutableList.of(a, b)) + .thenReturn(Collections.emptyList()); + when(secondIterator.readNextBatch()) + .thenReturn(singletonList(c)) + .thenReturn(singletonList(d)) + .thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + int recordsFound = 0; + while (recordsFound < 4) { + CustomOptional nextRecord = shardReadersPool.nextRecord(); + if (nextRecord.isPresent()) { + recordsFound++; + KinesisRecord kinesisRecord = nextRecord.get(); + if ("shard1".equals(kinesisRecord.getShardId())) { + verify(firstIterator).ackRecord(kinesisRecord); + } else { + verify(secondIterator).ackRecord(kinesisRecord); + } + } + } + } + + @Test + public void shouldInterruptKinesisReadingAndStopShortly() + throws TransientKinesisException, KinesisShardClosedException { + when(firstIterator.readNextBatch()) + .thenAnswer( + (Answer>) + invocation -> { + Thread.sleep(TIMEOUT_IN_MILLIS / 2); + return Collections.emptyList(); + }); + shardReadersPool.start(); + + Stopwatch stopwatch = Stopwatch.createStarted(); + shardReadersPool.stop(); + assertThat(stopwatch.elapsed(TimeUnit.MILLISECONDS)).isLessThan(TIMEOUT_IN_MILLIS); + } + + @Test + public void shouldInterruptPuttingRecordsToQueueAndStopShortly() + throws TransientKinesisException, KinesisShardClosedException { + when(firstIterator.readNextBatch()).thenReturn(ImmutableList.of(a, b, c)); + KinesisReaderCheckpoint checkpoint = + new KinesisReaderCheckpoint(ImmutableList.of(firstCheckpoint, secondCheckpoint)); + + WatermarkPolicyFactory watermarkPolicyFactory = WatermarkPolicyFactory.withArrivalTimePolicy(); + ShardReadersPool shardReadersPool = + new ShardReadersPool(kinesis, checkpoint, watermarkPolicyFactory, 2); + shardReadersPool.start(); + + Stopwatch stopwatch = Stopwatch.createStarted(); + shardReadersPool.stop(); + assertThat(stopwatch.elapsed(TimeUnit.MILLISECONDS)).isLessThan(TIMEOUT_IN_MILLIS); + } + + @Test + public void shouldStopReadingShardAfterReceivingShardClosedException() throws Exception { + when(firstIterator.readNextBatch()).thenThrow(KinesisShardClosedException.class); + when(firstIterator.findSuccessiveShardRecordIterators()).thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + + verify(firstIterator, timeout(TIMEOUT_IN_MILLIS).times(1)).readNextBatch(); + verify(secondIterator, timeout(TIMEOUT_IN_MILLIS).atLeast(2)).readNextBatch(); + } + + @Test + public void shouldStartReadingSuccessiveShardsAfterReceivingShardClosedException() + throws Exception { + when(firstIterator.readNextBatch()).thenThrow(KinesisShardClosedException.class); + when(firstIterator.findSuccessiveShardRecordIterators()) + .thenReturn(ImmutableList.of(thirdIterator, fourthIterator)); + + shardReadersPool.start(); + + verify(thirdIterator, timeout(TIMEOUT_IN_MILLIS).atLeast(2)).readNextBatch(); + verify(fourthIterator, timeout(TIMEOUT_IN_MILLIS).atLeast(2)).readNextBatch(); + } + + @Test + public void shouldStopReadersPoolWhenLastShardReaderStopped() throws Exception { + when(firstIterator.readNextBatch()).thenThrow(KinesisShardClosedException.class); + when(firstIterator.findSuccessiveShardRecordIterators()).thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + + verify(firstIterator, timeout(TIMEOUT_IN_MILLIS).times(1)).readNextBatch(); + } + + @Test + public void shouldStopReadersPoolAlsoWhenExceptionsOccurDuringStopping() throws Exception { + when(firstIterator.readNextBatch()).thenThrow(KinesisShardClosedException.class); + when(firstIterator.findSuccessiveShardRecordIterators()) + .thenThrow(TransientKinesisException.class) + .thenReturn(Collections.emptyList()); + + shardReadersPool.start(); + + verify(firstIterator, timeout(TIMEOUT_IN_MILLIS).times(2)).readNextBatch(); + } + + @Test + public void shouldReturnAbsentOptionalWhenStartedWithNoIterators() throws Exception { + KinesisReaderCheckpoint checkpoint = new KinesisReaderCheckpoint(Collections.emptyList()); + WatermarkPolicyFactory watermarkPolicyFactory = WatermarkPolicyFactory.withArrivalTimePolicy(); + shardReadersPool = + Mockito.spy(new ShardReadersPool(kinesis, checkpoint, watermarkPolicyFactory)); + doReturn(firstIterator) + .when(shardReadersPool) + .createShardIterator(eq(kinesis), any(ShardCheckpoint.class)); + + shardReadersPool.start(); + + assertThat(shardReadersPool.nextRecord()).isEqualTo(CustomOptional.absent()); + } + + @Test + public void shouldForgetClosedShardIterator() throws Exception { + when(firstIterator.readNextBatch()).thenThrow(KinesisShardClosedException.class); + List emptyList = Collections.emptyList(); + when(firstIterator.findSuccessiveShardRecordIterators()).thenReturn(emptyList); + + shardReadersPool.start(); + verify(shardReadersPool).startReadingShards(ImmutableList.of(firstIterator, secondIterator)); + verify(shardReadersPool, timeout(TIMEOUT_IN_MILLIS)).startReadingShards(emptyList); + + KinesisReaderCheckpoint checkpointMark = shardReadersPool.getCheckpointMark(); + assertThat(checkpointMark.iterator()) + .extracting("shardId", String.class) + .containsOnly("shard2") + .doesNotContain("shard1"); + } + + @Test + public void shouldReturnTheLeastWatermarkOfAllShards() throws TransientKinesisException { + Instant threeMin = now.minus(Duration.standardMinutes(3)); + Instant twoMin = now.minus(Duration.standardMinutes(2)); + + when(firstIterator.getShardWatermark()).thenReturn(threeMin).thenReturn(now); + when(secondIterator.getShardWatermark()).thenReturn(twoMin); + + shardReadersPool.start(); + + assertThat(shardReadersPool.getWatermark()).isEqualTo(threeMin); + assertThat(shardReadersPool.getWatermark()).isEqualTo(twoMin); + + verify(firstIterator, times(2)).getShardWatermark(); + verify(secondIterator, times(2)).getShardWatermark(); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java new file mode 100644 index 000000000000..ded3d3d168fe --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java @@ -0,0 +1,162 @@ +/* + * 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 static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Collections; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; + +/** Tests {@link ShardRecordsIterator}. */ +@RunWith(MockitoJUnitRunner.Silent.class) +public class ShardRecordsIteratorTest { + + private static final String INITIAL_ITERATOR = "INITIAL_ITERATOR"; + private static final String SECOND_ITERATOR = "SECOND_ITERATOR"; + private static final String SECOND_REFRESHED_ITERATOR = "SECOND_REFRESHED_ITERATOR"; + private static final String THIRD_ITERATOR = "THIRD_ITERATOR"; + private static final String STREAM_NAME = "STREAM_NAME"; + private static final String SHARD_ID = "SHARD_ID"; + private static final Instant NOW = Instant.now(); + + @Mock private SimplifiedKinesisClient kinesisClient; + @Mock private ShardCheckpoint firstCheckpoint, aCheckpoint, bCheckpoint, cCheckpoint, dCheckpoint; + @Mock private GetKinesisRecordsResult firstResult, secondResult, thirdResult; + @Mock private KinesisRecord a, b, c, d; + @Mock private RecordFilter recordFilter; + + private ShardRecordsIterator iterator; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(firstCheckpoint.getShardIterator(kinesisClient)).thenReturn(INITIAL_ITERATOR); + when(firstCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(firstCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(firstCheckpoint.moveAfter(a)).thenReturn(aCheckpoint); + when(aCheckpoint.moveAfter(b)).thenReturn(bCheckpoint); + when(aCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(aCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(bCheckpoint.moveAfter(c)).thenReturn(cCheckpoint); + when(bCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(bCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(cCheckpoint.moveAfter(d)).thenReturn(dCheckpoint); + when(cCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(cCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(dCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(dCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(kinesisClient.getRecords(INITIAL_ITERATOR, STREAM_NAME, SHARD_ID)).thenReturn(firstResult); + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)).thenReturn(secondResult); + when(kinesisClient.getRecords(THIRD_ITERATOR, STREAM_NAME, SHARD_ID)).thenReturn(thirdResult); + + when(firstResult.getNextShardIterator()).thenReturn(SECOND_ITERATOR); + when(secondResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + when(thirdResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + + when(firstResult.getRecords()).thenReturn(Collections.emptyList()); + when(secondResult.getRecords()).thenReturn(Collections.emptyList()); + when(thirdResult.getRecords()).thenReturn(Collections.emptyList()); + + when(recordFilter.apply(anyListOf(KinesisRecord.class), any(ShardCheckpoint.class))) + .thenAnswer(new IdentityAnswer()); + + WatermarkPolicyFactory watermarkPolicyFactory = WatermarkPolicyFactory.withArrivalTimePolicy(); + iterator = + new ShardRecordsIterator( + firstCheckpoint, kinesisClient, watermarkPolicyFactory, recordFilter); + } + + @Test + public void goesThroughAvailableRecords() + throws IOException, TransientKinesisException, KinesisShardClosedException { + when(firstResult.getRecords()).thenReturn(asList(a, b, c)); + when(secondResult.getRecords()).thenReturn(singletonList(d)); + when(thirdResult.getRecords()).thenReturn(Collections.emptyList()); + + assertThat(iterator.getCheckpoint()).isEqualTo(firstCheckpoint); + assertThat(iterator.readNextBatch()).isEqualTo(asList(a, b, c)); + assertThat(iterator.readNextBatch()).isEqualTo(singletonList(d)); + assertThat(iterator.readNextBatch()).isEqualTo(Collections.emptyList()); + } + + @Test + public void conformingRecordsMovesCheckpoint() throws IOException, TransientKinesisException { + when(firstResult.getRecords()).thenReturn(asList(a, b, c)); + when(secondResult.getRecords()).thenReturn(singletonList(d)); + when(thirdResult.getRecords()).thenReturn(Collections.emptyList()); + + when(a.getApproximateArrivalTimestamp()).thenReturn(NOW); + when(b.getApproximateArrivalTimestamp()).thenReturn(NOW.plus(Duration.standardSeconds(1))); + when(c.getApproximateArrivalTimestamp()).thenReturn(NOW.plus(Duration.standardSeconds(2))); + when(d.getApproximateArrivalTimestamp()).thenReturn(NOW.plus(Duration.standardSeconds(3))); + + iterator.ackRecord(a); + assertThat(iterator.getCheckpoint()).isEqualTo(aCheckpoint); + iterator.ackRecord(b); + assertThat(iterator.getCheckpoint()).isEqualTo(bCheckpoint); + iterator.ackRecord(c); + assertThat(iterator.getCheckpoint()).isEqualTo(cCheckpoint); + iterator.ackRecord(d); + assertThat(iterator.getCheckpoint()).isEqualTo(dCheckpoint); + } + + @Test + public void refreshesExpiredIterator() + throws IOException, TransientKinesisException, KinesisShardClosedException { + when(firstResult.getRecords()).thenReturn(singletonList(a)); + when(secondResult.getRecords()).thenReturn(singletonList(b)); + + when(a.getApproximateArrivalTimestamp()).thenReturn(NOW); + when(b.getApproximateArrivalTimestamp()).thenReturn(NOW.plus(Duration.standardSeconds(1))); + + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenThrow(ExpiredIteratorException.class); + when(aCheckpoint.getShardIterator(kinesisClient)).thenReturn(SECOND_REFRESHED_ITERATOR); + when(kinesisClient.getRecords(SECOND_REFRESHED_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(secondResult); + + assertThat(iterator.readNextBatch()).isEqualTo(singletonList(a)); + iterator.ackRecord(a); + assertThat(iterator.readNextBatch()).isEqualTo(singletonList(b)); + assertThat(iterator.readNextBatch()).isEqualTo(Collections.emptyList()); + } + + private static class IdentityAnswer implements Answer { + + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + return invocation.getArguments()[0]; + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java new file mode 100644 index 000000000000..12aee9f1186c --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java @@ -0,0 +1,377 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.joda.time.Instant; +import org.joda.time.Minutes; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.core.exception.SdkServiceException; +import software.amazon.awssdk.services.cloudwatch.CloudWatchClient; +import software.amazon.awssdk.services.cloudwatch.model.Datapoint; +import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsRequest; +import software.amazon.awssdk.services.cloudwatch.model.GetMetricStatisticsResponse; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; +import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse; +import software.amazon.awssdk.services.kinesis.model.LimitExceededException; +import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; + +/** * */ +@RunWith(MockitoJUnitRunner.class) +public class SimplifiedKinesisClientTest { + + private static final String STREAM = "stream"; + private static final String SHARD_1 = "shard-01"; + private static final String SHARD_2 = "shard-02"; + private static final String SHARD_3 = "shard-03"; + private static final String SHARD_ITERATOR = "iterator"; + private static final String SEQUENCE_NUMBER = "abc123"; + + @Mock private KinesisClient kinesis; + @Mock private KinesisAsyncClient kinesisAsync; + @Mock private CloudWatchClient cloudWatch; + @InjectMocks private SimplifiedKinesisClient underTest; + + @Test + public void shouldReturnIteratorStartingWithSequenceNumber() throws Exception { + when(kinesis.getShardIterator( + GetShardIteratorRequest.builder() + .streamName(STREAM) + .shardId(SHARD_1) + .shardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .startingSequenceNumber(SEQUENCE_NUMBER) + .build())) + .thenReturn(GetShardIteratorResponse.builder().shardIterator(SHARD_ITERATOR).build()); + + String stream = + underTest.getShardIterator( + STREAM, SHARD_1, ShardIteratorType.AT_SEQUENCE_NUMBER, SEQUENCE_NUMBER, null); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldReturnIteratorStartingWithTimestamp() throws Exception { + Instant timestamp = Instant.now(); + when(kinesis.getShardIterator( + GetShardIteratorRequest.builder() + .streamName(STREAM) + .shardId(SHARD_1) + .shardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .timestamp(TimeUtil.toJava(timestamp)) + .build())) + .thenReturn(GetShardIteratorResponse.builder().shardIterator(SHARD_ITERATOR).build()); + + String stream = + underTest.getShardIterator( + STREAM, SHARD_1, ShardIteratorType.AT_SEQUENCE_NUMBER, null, timestamp); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldHandleExpiredIterationExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError( + ExpiredIteratorException.builder().build(), ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError( + LimitExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError( + ProvisionedThroughputExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForGetShardIterator() { + shouldHandleGetShardIteratorError( + SdkServiceException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForGetShardIterator() { + shouldHandleGetShardIteratorError(SdkClientException.builder().build(), RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new NullPointerException(), RuntimeException.class); + } + + private void shouldHandleGetShardIteratorError( + Exception thrownException, Class expectedExceptionClass) { + GetShardIteratorRequest request = + GetShardIteratorRequest.builder() + .streamName(STREAM) + .shardId(SHARD_1) + .shardIteratorType(ShardIteratorType.LATEST) + .build(); + + when(kinesis.getShardIterator(request)).thenThrow(thrownException); + + try { + underTest.getShardIterator(STREAM, SHARD_1, ShardIteratorType.LATEST, null, null); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + @Test + public void shouldListAllShards() throws Exception { + Shard shard1 = Shard.builder().shardId(SHARD_1).build(); + Shard shard2 = Shard.builder().shardId(SHARD_2).build(); + Shard shard3 = Shard.builder().shardId(SHARD_3).build(); + when(kinesis.describeStream( + DescribeStreamRequest.builder().streamName(STREAM).exclusiveStartShardId(null).build())) + .thenReturn( + DescribeStreamResponse.builder() + .streamDescription(s -> s.shards(shard1, shard2).hasMoreShards(true)) + .build()); + when(kinesis.describeStream( + DescribeStreamRequest.builder() + .streamName(STREAM) + .exclusiveStartShardId(SHARD_2) + .build())) + .thenReturn( + DescribeStreamResponse.builder() + .streamDescription(s -> s.shards(shard3).hasMoreShards(false)) + .build()); + + List shards = underTest.listShards(STREAM); + + assertThat(shards).containsOnly(shard1, shard2, shard3); + } + + @Test + public void shouldHandleExpiredIterationExceptionForShardListing() { + shouldHandleShardListingError( + ExpiredIteratorException.builder().build(), ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForShardListing() { + shouldHandleShardListingError( + LimitExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForShardListing() { + shouldHandleShardListingError( + ProvisionedThroughputExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForShardListing() { + shouldHandleShardListingError( + SdkServiceException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForShardListing() { + shouldHandleShardListingError(SdkClientException.builder().build(), RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForShardListing() { + shouldHandleShardListingError(new NullPointerException(), RuntimeException.class); + } + + private void shouldHandleShardListingError( + Exception thrownException, Class expectedExceptionClass) { + when(kinesis.describeStream(any(DescribeStreamRequest.class))).thenThrow(thrownException); + try { + underTest.listShards(STREAM); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + @Test + public void shouldCountBytesWhenSingleDataPointReturned() throws Exception { + Instant countSince = new Instant("2017-04-06T10:00:00.000Z"); + Instant countTo = new Instant("2017-04-06T11:00:00.000Z"); + Minutes periodTime = Minutes.minutesBetween(countSince, countTo); + GetMetricStatisticsRequest metricStatisticsRequest = + underTest.createMetricStatisticsRequest(STREAM, countSince, countTo, periodTime); + GetMetricStatisticsResponse result = + GetMetricStatisticsResponse.builder() + .datapoints(Datapoint.builder().sum(1.0).build()) + .build(); + + when(cloudWatch.getMetricStatistics(metricStatisticsRequest)).thenReturn(result); + + long backlogBytes = underTest.getBacklogBytes(STREAM, countSince, countTo); + + assertThat(backlogBytes).isEqualTo(1L); + } + + @Test + public void shouldCountBytesWhenMultipleDataPointsReturned() throws Exception { + Instant countSince = new Instant("2017-04-06T10:00:00.000Z"); + Instant countTo = new Instant("2017-04-06T11:00:00.000Z"); + Minutes periodTime = Minutes.minutesBetween(countSince, countTo); + GetMetricStatisticsRequest metricStatisticsRequest = + underTest.createMetricStatisticsRequest(STREAM, countSince, countTo, periodTime); + GetMetricStatisticsResponse result = + GetMetricStatisticsResponse.builder() + .datapoints( + Datapoint.builder().sum(1.0).build(), + Datapoint.builder().sum(3.0).build(), + Datapoint.builder().sum(2.0).build()) + .build(); + + when(cloudWatch.getMetricStatistics(metricStatisticsRequest)).thenReturn(result); + + long backlogBytes = underTest.getBacklogBytes(STREAM, countSince, countTo); + + assertThat(backlogBytes).isEqualTo(6L); + } + + @Test + public void shouldNotCallCloudWatchWhenSpecifiedPeriodTooShort() throws Exception { + Instant countSince = new Instant("2017-04-06T10:00:00.000Z"); + Instant countTo = new Instant("2017-04-06T10:00:02.000Z"); + + long backlogBytes = underTest.getBacklogBytes(STREAM, countSince, countTo); + + assertThat(backlogBytes).isEqualTo(0L); + verifyZeroInteractions(cloudWatch); + } + + @Test + public void shouldHandleLimitExceededExceptionForGetBacklogBytes() { + shouldHandleGetBacklogBytesError( + LimitExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForGetBacklogBytes() { + shouldHandleGetBacklogBytesError( + ProvisionedThroughputExceededException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForGetBacklogBytes() { + shouldHandleGetBacklogBytesError( + SdkServiceException.builder().build(), TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForGetBacklogBytes() { + shouldHandleGetBacklogBytesError(SdkClientException.builder().build(), RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForGetBacklogBytes() { + shouldHandleGetBacklogBytesError(new NullPointerException(), RuntimeException.class); + } + + private void shouldHandleGetBacklogBytesError( + Exception thrownException, Class expectedExceptionClass) { + Instant countSince = new Instant("2017-04-06T10:00:00.000Z"); + Instant countTo = new Instant("2017-04-06T11:00:00.000Z"); + Minutes periodTime = Minutes.minutesBetween(countSince, countTo); + GetMetricStatisticsRequest metricStatisticsRequest = + underTest.createMetricStatisticsRequest(STREAM, countSince, countTo, periodTime); + + when(cloudWatch.getMetricStatistics(metricStatisticsRequest)).thenThrow(thrownException); + try { + underTest.getBacklogBytes(STREAM, countSince, countTo); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + @Test + public void shouldReturnLimitedNumberOfRecords() throws Exception { + final Integer limit = 100; + + doAnswer( + (Answer) + invocation -> { + GetRecordsRequest request = (GetRecordsRequest) invocation.getArguments()[0]; + List records = generateRecords(request.limit()); + return GetRecordsResponse.builder() + .records(records) + .millisBehindLatest(1000L) + .build(); + }) + .when(kinesis) + .getRecords(any(GetRecordsRequest.class)); + + GetKinesisRecordsResult result = underTest.getRecords(SHARD_ITERATOR, STREAM, SHARD_1, limit); + assertThat(result.getRecords().size()).isEqualTo(limit); + } + + private List generateRecords(int num) { + List records = new ArrayList<>(); + for (int i = 0; i < num; i++) { + byte[] value = new byte[1024]; + Arrays.fill(value, (byte) i); + records.add( + Record.builder() + .sequenceNumber(String.valueOf(i)) + .partitionKey("key") + .data(SdkBytes.fromByteBuffer(ByteBuffer.wrap(value))) + .build()); + } + return records; + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java new file mode 100644 index 000000000000..51d97bb3760b --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java @@ -0,0 +1,292 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.Collections; +import java.util.List; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +/** Tests StartingPointShardsFinder. */ +@RunWith(JUnit4.class) +public class StartingPointShardsFinderTest { + + private static final String STREAM_NAME = "streamName"; + private SimplifiedKinesisClient kinesis = mock(SimplifiedKinesisClient.class); + + /* + * This test operates on shards hierarchy prepared upfront. + * Following diagram depicts shard split and merge operations: + * + * 0002------------------------------+ + * / \ + * 0000------+ 0009 + * \ / + * 0003------+ 0007------+ + * \ / + * 0006------+ + * / \ + * 0004------+ 0008------+ + * / \ + * 0001------+ 0010 + * \ / + * 0005------------------------------+ + * + */ + private final Shard shard00 = createClosedShard("0000"); + private final Shard shard01 = createClosedShard("0001"); + private final Shard shard02 = createClosedShard("0002").toBuilder().parentShardId("0000").build(); + private final Shard shard03 = createClosedShard("0003").toBuilder().parentShardId("0000").build(); + private final Shard shard04 = createClosedShard("0004").toBuilder().parentShardId("0001").build(); + private final Shard shard05 = createClosedShard("0005").toBuilder().parentShardId("0001").build(); + private final Shard shard06 = + createClosedShard("0006") + .toBuilder() + .parentShardId("0003") + .adjacentParentShardId("0004") + .build(); + private final Shard shard07 = createClosedShard("0007").toBuilder().parentShardId("0006").build(); + private final Shard shard08 = createClosedShard("0008").toBuilder().parentShardId("0006").build(); + private final Shard shard09 = + createOpenShard("0009") + .toBuilder() + .parentShardId("0002") + .adjacentParentShardId("0007") + .build(); + private final Shard shard10 = + createOpenShard("0010") + .toBuilder() + .parentShardId("0008") + .adjacentParentShardId("0005") + .build(); + + private final List allShards = + ImmutableList.of( + shard00, shard01, shard02, shard03, shard04, shard05, shard06, shard07, shard08, shard09, + shard10); + + private StartingPointShardsFinder underTest = new StartingPointShardsFinder(); + + @Test + public void shouldFindFirstShardsWhenAllShardsAreValid() throws Exception { + // given + Instant timestampAtTheBeginning = new Instant(); + StartingPoint startingPointAtTheBeginning = new StartingPoint(timestampAtTheBeginning); + for (Shard shard : allShards) { + activeAtTimestamp(shard, timestampAtTheBeginning); + } + when(kinesis.listShards(STREAM_NAME)).thenReturn(allShards); + + // when + Iterable shardsAtStartingPoint = + underTest.findShardsAtStartingPoint(kinesis, STREAM_NAME, startingPointAtTheBeginning); + + // then + assertThat(shardsAtStartingPoint).containsExactlyInAnyOrder(shard00, shard01); + } + + @Test + public void shouldFind3StartingShardsInTheMiddle() throws Exception { + // given + Instant timestampAfterShards3And4Merge = new Instant(); + StartingPoint startingPointAfterFirstSplitsAndMerge = + new StartingPoint(timestampAfterShards3And4Merge); + + expiredAtTimestamp(shard00, timestampAfterShards3And4Merge); + expiredAtTimestamp(shard01, timestampAfterShards3And4Merge); + activeAtTimestamp(shard02, timestampAfterShards3And4Merge); + expiredAtTimestamp(shard03, timestampAfterShards3And4Merge); + expiredAtTimestamp(shard04, timestampAfterShards3And4Merge); + activeAtTimestamp(shard05, timestampAfterShards3And4Merge); + activeAtTimestamp(shard06, timestampAfterShards3And4Merge); + activeAtTimestamp(shard07, timestampAfterShards3And4Merge); + activeAtTimestamp(shard08, timestampAfterShards3And4Merge); + activeAtTimestamp(shard09, timestampAfterShards3And4Merge); + activeAtTimestamp(shard10, timestampAfterShards3And4Merge); + + when(kinesis.listShards(STREAM_NAME)).thenReturn(allShards); + + // when + Iterable shardsAtStartingPoint = + underTest.findShardsAtStartingPoint( + kinesis, STREAM_NAME, startingPointAfterFirstSplitsAndMerge); + + // then + assertThat(shardsAtStartingPoint).containsExactlyInAnyOrder(shard02, shard05, shard06); + } + + @Test + public void shouldFindLastShardWhenAllPreviousExpired() throws Exception { + // given + Instant timestampAtTheEnd = new Instant(); + StartingPoint startingPointAtTheEnd = new StartingPoint(timestampAtTheEnd); + + expiredAtTimestamp(shard00, timestampAtTheEnd); + expiredAtTimestamp(shard01, timestampAtTheEnd); + expiredAtTimestamp(shard02, timestampAtTheEnd); + expiredAtTimestamp(shard03, timestampAtTheEnd); + expiredAtTimestamp(shard04, timestampAtTheEnd); + expiredAtTimestamp(shard05, timestampAtTheEnd); + expiredAtTimestamp(shard06, timestampAtTheEnd); + expiredAtTimestamp(shard07, timestampAtTheEnd); + expiredAtTimestamp(shard08, timestampAtTheEnd); + activeAtTimestamp(shard09, timestampAtTheEnd); + activeAtTimestamp(shard10, timestampAtTheEnd); + + when(kinesis.listShards(STREAM_NAME)).thenReturn(allShards); + + // when + Iterable shardsAtStartingPoint = + underTest.findShardsAtStartingPoint(kinesis, STREAM_NAME, startingPointAtTheEnd); + + // then + assertThat(shardsAtStartingPoint).containsExactlyInAnyOrder(shard09, shard10); + } + + @Test + public void shouldFindLastShardsWhenLatestStartingPointRequested() throws Exception { + // given + StartingPoint latestStartingPoint = new StartingPoint(InitialPositionInStream.LATEST); + when(kinesis.listShards(STREAM_NAME)).thenReturn(allShards); + + // when + Iterable shardsAtStartingPoint = + underTest.findShardsAtStartingPoint(kinesis, STREAM_NAME, latestStartingPoint); + + // then + assertThat(shardsAtStartingPoint).containsExactlyInAnyOrder(shard09, shard10); + } + + @Test + public void shouldFindEarliestShardsWhenTrimHorizonStartingPointRequested() throws Exception { + // given + StartingPoint trimHorizonStartingPoint = + new StartingPoint(InitialPositionInStream.TRIM_HORIZON); + when(kinesis.listShards(STREAM_NAME)).thenReturn(allShards); + + // when + Iterable shardsAtStartingPoint = + underTest.findShardsAtStartingPoint(kinesis, STREAM_NAME, trimHorizonStartingPoint); + + // then + assertThat(shardsAtStartingPoint).containsExactlyInAnyOrder(shard00, shard01); + } + + @Test(expected = IllegalStateException.class) + public void shouldThrowExceptionWhenSuccessorsNotFoundForExpiredShard() throws Exception { + // given + StartingPoint latestStartingPoint = new StartingPoint(InitialPositionInStream.LATEST); + Shard closedShard10 = + createClosedShard("0010") + .toBuilder() + .parentShardId("0008") + .adjacentParentShardId("0005") + .build(); + List shards = + ImmutableList.of( + shard00, + shard01, + shard02, + shard03, + shard04, + shard05, + shard06, + shard07, + shard08, + shard09, + closedShard10); + + when(kinesis.listShards(STREAM_NAME)).thenReturn(shards); + + // when + underTest.findShardsAtStartingPoint(kinesis, STREAM_NAME, latestStartingPoint); + } + + private Shard createClosedShard(String shardId) { + Shard shard = Shard.builder().shardId(shardId).build(); + activeAtPoint(shard, ShardIteratorType.TRIM_HORIZON); + expiredAtPoint(shard, ShardIteratorType.LATEST); + return shard; + } + + private Shard createOpenShard(String shardId) { + Shard shard = Shard.builder().shardId(shardId).build(); + activeAtPoint(shard, ShardIteratorType.TRIM_HORIZON); + activeAtPoint(shard, ShardIteratorType.LATEST); + return shard; + } + + private void expiredAtTimestamp(Shard shard, Instant startTimestamp) { + prepareShard(shard, null, ShardIteratorType.AT_TIMESTAMP, startTimestamp); + } + + private void expiredAtPoint(Shard shard, ShardIteratorType shardIteratorType) { + prepareShard(shard, null, shardIteratorType, null); + } + + private void activeAtTimestamp(Shard shard, Instant startTimestamp) { + prepareShard( + shard, + "timestampIterator-" + shard.shardId(), + ShardIteratorType.AT_TIMESTAMP, + startTimestamp); + } + + private void activeAtPoint(Shard shard, ShardIteratorType shardIteratorType) { + prepareShard(shard, shardIteratorType.toString() + shard.shardId(), shardIteratorType, null); + } + + private void prepareShard( + Shard shard, + String nextIterator, + ShardIteratorType shardIteratorType, + Instant startTimestamp) { + try { + String shardIterator = shardIteratorType + shard.shardId() + "-current"; + if (shardIteratorType == ShardIteratorType.AT_TIMESTAMP) { + when(kinesis.getShardIterator( + STREAM_NAME, shard.shardId(), ShardIteratorType.AT_TIMESTAMP, null, startTimestamp)) + .thenReturn(shardIterator); + } else { + when(kinesis.getShardIterator(STREAM_NAME, shard.shardId(), shardIteratorType, null, null)) + .thenReturn(shardIterator); + } + GetKinesisRecordsResult result = + new GetKinesisRecordsResult( + Collections.emptyList(), + nextIterator, + 0, + STREAM_NAME, + shard.shardId()); + when(kinesis.getRecords(shardIterator, STREAM_NAME, shard.shardId())).thenReturn(result); + } catch (TransientKinesisException e) { + throw new RuntimeException(e); + } + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyTest.java new file mode 100644 index 000000000000..08d5b8b7939a --- /dev/null +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/WatermarkPolicyTest.java @@ -0,0 +1,152 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mockStatic; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +/** Tests {@link WatermarkPolicy}. */ +@RunWith(PowerMockRunner.class) +@PrepareForTest(Instant.class) +public class WatermarkPolicyTest { + private static final Instant NOW = Instant.now(); + + @Test + public void shouldAdvanceWatermarkWithTheArrivalTimeFromKinesisRecords() { + WatermarkPolicy policy = WatermarkPolicyFactory.withArrivalTimePolicy().createWatermarkPolicy(); + + KinesisRecord a = mock(KinesisRecord.class); + KinesisRecord b = mock(KinesisRecord.class); + + Instant time1 = NOW.minus(Duration.standardSeconds(30L)); + Instant time2 = NOW.minus(Duration.standardSeconds(20L)); + when(a.getApproximateArrivalTimestamp()).thenReturn(time1); + when(b.getApproximateArrivalTimestamp()).thenReturn(time2); + + policy.update(a); + assertThat(policy.getWatermark()).isEqualTo(time1); + policy.update(b); + assertThat(policy.getWatermark()).isEqualTo(time2); + } + + @Test + public void shouldOnlyAdvanceTheWatermark() { + WatermarkPolicy policy = WatermarkPolicyFactory.withArrivalTimePolicy().createWatermarkPolicy(); + + KinesisRecord a = mock(KinesisRecord.class); + KinesisRecord b = mock(KinesisRecord.class); + KinesisRecord c = mock(KinesisRecord.class); + + Instant time1 = NOW.minus(Duration.standardSeconds(30L)); + Instant time2 = NOW.minus(Duration.standardSeconds(20L)); + Instant time3 = NOW.minus(Duration.standardSeconds(40L)); + when(a.getApproximateArrivalTimestamp()).thenReturn(time1); + when(b.getApproximateArrivalTimestamp()).thenReturn(time2); + // time3 is before time2 + when(c.getApproximateArrivalTimestamp()).thenReturn(time3); + + policy.update(a); + assertThat(policy.getWatermark()).isEqualTo(time1); + policy.update(b); + assertThat(policy.getWatermark()).isEqualTo(time2); + policy.update(c); + // watermark doesn't go back in time + assertThat(policy.getWatermark()).isEqualTo(time2); + } + + @Test + public void shouldAdvanceWatermarkWhenThereAreNoIncomingRecords() { + WatermarkParameters standardWatermarkParams = WatermarkParameters.create(); + WatermarkPolicy policy = + WatermarkPolicyFactory.withCustomWatermarkPolicy(standardWatermarkParams) + .createWatermarkPolicy(); + + mockStatic(Instant.class); + + Instant time1 = NOW.minus(Duration.standardSeconds(500)); // returned when update is called + Instant time2 = + NOW.minus( + Duration.standardSeconds(498)); // returned when getWatermark is called the first time + Instant time3 = NOW; // returned when getWatermark is called the second time + Instant arrivalTime = NOW.minus(Duration.standardSeconds(510)); + Duration watermarkIdleTimeThreshold = + standardWatermarkParams.getWatermarkIdleDurationThreshold(); + + when(Instant.now()).thenReturn(time1).thenReturn(time2).thenReturn(time3); + + KinesisRecord a = mock(KinesisRecord.class); + when(a.getApproximateArrivalTimestamp()).thenReturn(arrivalTime); + + policy.update(a); + + // returns the latest event time when the watermark + assertThat(policy.getWatermark()).isEqualTo(arrivalTime); + // advance the watermark to [NOW - watermark idle time threshold] + assertThat(policy.getWatermark()).isEqualTo(time3.minus(watermarkIdleTimeThreshold)); + } + + @Test + public void shouldAdvanceWatermarkToNowWithProcessingTimePolicy() { + WatermarkPolicy policy = + WatermarkPolicyFactory.withProcessingTimePolicy().createWatermarkPolicy(); + + mockStatic(Instant.class); + + Instant time1 = NOW.minus(Duration.standardSeconds(5)); + Instant time2 = NOW.minus(Duration.standardSeconds(4)); + + when(Instant.now()).thenReturn(time1).thenReturn(time2); + + assertThat(policy.getWatermark()).isEqualTo(time1); + assertThat(policy.getWatermark()).isEqualTo(time2); + } + + @Test + public void shouldAdvanceWatermarkWithCustomTimePolicy() { + SerializableFunction timestampFn = + (record) -> record.getApproximateArrivalTimestamp().plus(Duration.standardMinutes(1)); + + WatermarkPolicy policy = + WatermarkPolicyFactory.withCustomWatermarkPolicy( + WatermarkParameters.create().withTimestampFn(timestampFn)) + .createWatermarkPolicy(); + + KinesisRecord a = mock(KinesisRecord.class); + KinesisRecord b = mock(KinesisRecord.class); + + Instant time1 = NOW.minus(Duration.standardSeconds(30L)); + Instant time2 = NOW.minus(Duration.standardSeconds(20L)); + when(a.getApproximateArrivalTimestamp()).thenReturn(time1); + when(b.getApproximateArrivalTimestamp()).thenReturn(time2); + + policy.update(a); + assertThat(policy.getWatermark()).isEqualTo(time1.plus(Duration.standardMinutes(1))); + policy.update(b); + assertThat(policy.getWatermark()).isEqualTo(time2.plus(Duration.standardMinutes(1))); + } +} diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java index e9372dcd12df..9aae09ef8eef 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java @@ -32,12 +32,15 @@ import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import software.amazon.awssdk.services.sns.model.GetTopicAttributesResponse; import software.amazon.awssdk.services.sns.model.PublishRequest; import software.amazon.awssdk.services.sns.model.PublishResponse; /** Tests to verify writes to Sns. */ -@RunWith(JUnit4.class) +@RunWith(PowerMockRunner.class) +@PrepareForTest({PublishResponse.class, GetTopicAttributesResponse.class}) public class SnsIOTest implements Serializable { private static final String topicArn = "arn:aws:sns:us-west-2:5880:topic-FMFEHJ47NRFO"; diff --git a/sdks/java/io/amazon-web-services2/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/sdks/java/io/amazon-web-services2/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 1f0955d450f0..000000000000 --- a/sdks/java/io/amazon-web-services2/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1 +0,0 @@ -mock-maker-inline