You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "mosche (via GitHub)" <gi...@apache.org> on 2023/04/11 09:31:48 UTC

[GitHub] [beam] mosche commented on a diff in pull request #26117: Refactor Kinesis IO

mosche commented on code in PR #26117:
URL: https://github.com/apache/beam/pull/26117#discussion_r1162446821


##########
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardListingUtils.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.util.List;
+import java.util.concurrent.Callable;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.BackOffUtils;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Sleeper;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.core.exception.SdkServiceException;
+import software.amazon.awssdk.core.internal.retry.SdkDefaultRetrySetting;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest;
+import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
+import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardFilter;
+import software.amazon.awssdk.services.kinesis.model.ShardFilterType;
+import software.amazon.awssdk.services.kinesis.model.StreamDescriptionSummary;
+import software.amazon.kinesis.common.InitialPositionInStream;
+
+class ShardListingUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(ShardListingUtils.class);
+
+  private static final int LIST_SHARDS_MAX_RESULTS = 1_000;
+  private static final Duration
+      SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON =
+          Duration.standardMinutes(5);
+  private static final int DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS = 10;
+  private static final Duration DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF =
+      Duration.standardSeconds(1);
+
+  static List<Shard> listShardsAtPoint(
+      KinesisClient kinesisClient, final String streamName, final StartingPoint startingPoint)
+      throws TransientKinesisException {
+    ShardFilter shardFilter =
+        wrapExceptions(
+            () -> buildShardFilterForStartingPoint(kinesisClient, streamName, startingPoint));
+    return listShards(kinesisClient, streamName, shardFilter);
+  }
+
+  static ShardFilter buildShardFilterForStartingPoint(
+      KinesisClient kinesisClient, String streamName, StartingPoint startingPoint)
+      throws IOException, InterruptedException {
+    InitialPositionInStream position = startingPoint.getPosition();
+    switch (position) {
+      case LATEST:
+        return ShardFilter.builder().type(ShardFilterType.AT_LATEST).build();
+      case TRIM_HORIZON:
+        return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+      case AT_TIMESTAMP:
+        return buildShardFilterForTimestamp(
+            kinesisClient, streamName, startingPoint.getTimestamp());
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unrecognized '%s' position to create shard filter with", position));
+    }
+  }
+
+  private static ShardFilter buildShardFilterForTimestamp(
+      KinesisClient kinesisClient, String streamName, Instant startingPointTimestamp)
+      throws IOException, InterruptedException {
+    StreamDescriptionSummary streamDescription = describeStreamSummary(kinesisClient, streamName);
+
+    Instant streamCreationTimestamp = TimeUtil.toJoda(streamDescription.streamCreationTimestamp());
+    if (streamCreationTimestamp.isAfter(startingPointTimestamp)) {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+
+    Duration retentionPeriod = Duration.standardHours(streamDescription.retentionPeriodHours());
+
+    Instant streamTrimHorizonTimestamp =
+        Instant.now()
+            .minus(retentionPeriod)
+            .plus(SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON);
+    if (startingPointTimestamp.isAfter(streamTrimHorizonTimestamp)) {
+      return ShardFilter.builder()
+          .type(ShardFilterType.AT_TIMESTAMP)
+          .timestamp(TimeUtil.toJava(startingPointTimestamp))
+          .build();
+    } else {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+  }
+
+  private static StreamDescriptionSummary describeStreamSummary(
+      KinesisClient kinesisClient, final String streamName)
+      throws IOException, InterruptedException {
+    // DescribeStreamSummary has limits that can be hit fairly easily if we are attempting
+    // to configure multiple KinesisIO inputs in the same account. Retry up to
+    // DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS times if we end up hitting that limit.
+    //
+    // Only pass the wrapped exception up once that limit is reached. Use FluentBackoff
+    // to implement the retry policy.
+    FluentBackoff retryBackoff =
+        FluentBackoff.DEFAULT
+            .withMaxRetries(DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS)
+            .withInitialBackoff(DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF);
+    BackOff backoff = retryBackoff.backoff();
+    Sleeper sleeper = Sleeper.DEFAULT;
+
+    DescribeStreamSummaryRequest request =
+        DescribeStreamSummaryRequest.builder().streamName(streamName).build();
+    while (true) {
+      try {
+        LOG.info("Executing request: {}", request);
+        return kinesisClient.describeStreamSummary(request).streamDescriptionSummary();
+      } catch (LimitExceededException exc) {
+        if (!BackOffUtils.next(sleeper, backoff)) {
+          throw exc;
+        }
+      }
+    }
+  }
+
+  static List<Shard> listShards(
+      KinesisClient kinesisClient, final String streamName, final ShardFilter shardFilter)
+      throws TransientKinesisException {
+    return wrapExceptions(
+        () -> {
+          ImmutableList.Builder<Shard> shardsBuilder = ImmutableList.builder();
+
+          String currentNextToken = null;
+          do {
+            ListShardsRequest.Builder reqBuilder =
+                ListShardsRequest.builder()
+                    .maxResults(LIST_SHARDS_MAX_RESULTS)
+                    .shardFilter(shardFilter);
+            if (currentNextToken != null) {
+              reqBuilder.nextToken(currentNextToken);
+            } else {
+              reqBuilder.streamName(streamName);
+            }
+
+            ListShardsRequest request = reqBuilder.build();
+            LOG.info("Executing request: {}", request);
+            ListShardsResponse response = kinesisClient.listShards(request);
+            shardsBuilder.addAll(response.shards());
+            currentNextToken = response.nextToken();
+          } while (currentNextToken != null);
+
+          return shardsBuilder.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 static <T> T wrapExceptions(Callable<T> callable) throws TransientKinesisException {

Review Comment:
   No need to copy this / wrap exceptions this way for the new implementation. These custom exceptions are not used anywhere.
   Just make sure you use `wrapExceptions(() -> ShardListingUtils.listShards/listShardsAtPoint(...))` where used in SimplifiedKinesisClient.



##########
sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/Helpers.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.IntStream.range;
+import static org.joda.time.Duration.standardSeconds;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.mockito.ArgumentMatcher;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+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.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+class Helpers {

Review Comment:
   Nitpick, maybe `TestHelpers`?



##########
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardListingUtils.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.util.List;
+import java.util.concurrent.Callable;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.BackOffUtils;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Sleeper;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.core.exception.SdkServiceException;
+import software.amazon.awssdk.core.internal.retry.SdkDefaultRetrySetting;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest;
+import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
+import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardFilter;
+import software.amazon.awssdk.services.kinesis.model.ShardFilterType;
+import software.amazon.awssdk.services.kinesis.model.StreamDescriptionSummary;
+import software.amazon.kinesis.common.InitialPositionInStream;
+
+class ShardListingUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(ShardListingUtils.class);
+
+  private static final int LIST_SHARDS_MAX_RESULTS = 1_000;
+  private static final Duration
+      SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON =
+          Duration.standardMinutes(5);
+  private static final int DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS = 10;
+  private static final Duration DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF =
+      Duration.standardSeconds(1);
+
+  static List<Shard> listShardsAtPoint(
+      KinesisClient kinesisClient, final String streamName, final StartingPoint startingPoint)
+      throws TransientKinesisException {
+    ShardFilter shardFilter =
+        wrapExceptions(
+            () -> buildShardFilterForStartingPoint(kinesisClient, streamName, startingPoint));
+    return listShards(kinesisClient, streamName, shardFilter);
+  }
+
+  static ShardFilter buildShardFilterForStartingPoint(
+      KinesisClient kinesisClient, String streamName, StartingPoint startingPoint)
+      throws IOException, InterruptedException {
+    InitialPositionInStream position = startingPoint.getPosition();
+    switch (position) {
+      case LATEST:
+        return ShardFilter.builder().type(ShardFilterType.AT_LATEST).build();
+      case TRIM_HORIZON:
+        return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+      case AT_TIMESTAMP:
+        return buildShardFilterForTimestamp(
+            kinesisClient, streamName, startingPoint.getTimestamp());
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unrecognized '%s' position to create shard filter with", position));
+    }
+  }
+
+  private static ShardFilter buildShardFilterForTimestamp(
+      KinesisClient kinesisClient, String streamName, Instant startingPointTimestamp)
+      throws IOException, InterruptedException {
+    StreamDescriptionSummary streamDescription = describeStreamSummary(kinesisClient, streamName);
+
+    Instant streamCreationTimestamp = TimeUtil.toJoda(streamDescription.streamCreationTimestamp());
+    if (streamCreationTimestamp.isAfter(startingPointTimestamp)) {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+
+    Duration retentionPeriod = Duration.standardHours(streamDescription.retentionPeriodHours());
+
+    Instant streamTrimHorizonTimestamp =
+        Instant.now()
+            .minus(retentionPeriod)
+            .plus(SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON);
+    if (startingPointTimestamp.isAfter(streamTrimHorizonTimestamp)) {
+      return ShardFilter.builder()
+          .type(ShardFilterType.AT_TIMESTAMP)
+          .timestamp(TimeUtil.toJava(startingPointTimestamp))
+          .build();
+    } else {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+  }
+
+  private static StreamDescriptionSummary describeStreamSummary(
+      KinesisClient kinesisClient, final String streamName)
+      throws IOException, InterruptedException {
+    // DescribeStreamSummary has limits that can be hit fairly easily if we are attempting
+    // to configure multiple KinesisIO inputs in the same account. Retry up to
+    // DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS times if we end up hitting that limit.
+    //
+    // Only pass the wrapped exception up once that limit is reached. Use FluentBackoff
+    // to implement the retry policy.
+    FluentBackoff retryBackoff =
+        FluentBackoff.DEFAULT
+            .withMaxRetries(DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS)
+            .withInitialBackoff(DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF);
+    BackOff backoff = retryBackoff.backoff();
+    Sleeper sleeper = Sleeper.DEFAULT;
+
+    DescribeStreamSummaryRequest request =
+        DescribeStreamSummaryRequest.builder().streamName(streamName).build();
+    while (true) {
+      try {
+        LOG.info("Executing request: {}", request);

Review Comment:
   Change to debug pls



##########
sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/Helpers.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.IntStream.range;
+import static org.joda.time.Duration.standardSeconds;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.mockito.ArgumentMatcher;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+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.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+class Helpers {
+  static final int SHARD_EVENTS = 100;
+
+  static void mockShards(KinesisClient client, int count) {
+    IntFunction<Shard> shard = i -> Shard.builder().shardId(Integer.toString(i)).build();
+    List<Shard> shards = range(0, count).mapToObj(shard).collect(toList());
+    when(client.listShards(any(ListShardsRequest.class)))
+        .thenReturn(ListShardsResponse.builder().shards(shards).build());
+  }
+
+  static void mockShardIterators(KinesisClient client, List<List<Record>> data) {
+    for (int id = 0; id < data.size(); id++) {
+      when(client.getShardIterator(argThat(hasShardId(id))))
+          .thenReturn(GetShardIteratorResponse.builder().shardIterator(id + ":0").build());
+    }
+  }
+
+  static void mockRecords(KinesisClient client, List<List<Record>> data, int limit) {
+    BiFunction<List<Record>, String, GetRecordsResponse.Builder> resp =
+        (recs, it) ->
+            GetRecordsResponse.builder().millisBehindLatest(0L).records(recs).nextShardIterator(it);
+
+    for (int shard = 0; shard < data.size(); shard++) {
+      List<Record> records = data.get(shard);
+      for (int i = 0; i < records.size(); i += limit) {
+        int to = Math.max(i + limit, records.size());
+        String nextIt = (to == records.size()) ? "done" : shard + ":" + to;
+        when(client.getRecords(argThat(hasShardIterator(shard + ":" + i))))
+            .thenReturn(resp.apply(records.subList(i, to), nextIt).build());
+      }
+    }
+    when(client.getRecords(argThat(hasShardIterator("done"))))
+        .thenReturn(resp.apply(ImmutableList.of(), "done").build());
+  }
+
+  static List<List<Record>> testRecords(int shards, int events) {

Review Comment:
   Nitpick, `createRecords`



##########
sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardListingUtils.java:
##########
@@ -0,0 +1,203 @@
+/*
+ * 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.util.List;
+import java.util.concurrent.Callable;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.BackOffUtils;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.util.Sleeper;
+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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.core.exception.SdkServiceException;
+import software.amazon.awssdk.core.internal.retry.SdkDefaultRetrySetting;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest;
+import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException;
+import software.amazon.awssdk.services.kinesis.model.LimitExceededException;
+import software.amazon.awssdk.services.kinesis.model.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.ProvisionedThroughputExceededException;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+import software.amazon.awssdk.services.kinesis.model.ShardFilter;
+import software.amazon.awssdk.services.kinesis.model.ShardFilterType;
+import software.amazon.awssdk.services.kinesis.model.StreamDescriptionSummary;
+import software.amazon.kinesis.common.InitialPositionInStream;
+
+class ShardListingUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(ShardListingUtils.class);
+
+  private static final int LIST_SHARDS_MAX_RESULTS = 1_000;
+  private static final Duration
+      SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON =
+          Duration.standardMinutes(5);
+  private static final int DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS = 10;
+  private static final Duration DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF =
+      Duration.standardSeconds(1);
+
+  static List<Shard> listShardsAtPoint(
+      KinesisClient kinesisClient, final String streamName, final StartingPoint startingPoint)
+      throws TransientKinesisException {
+    ShardFilter shardFilter =
+        wrapExceptions(
+            () -> buildShardFilterForStartingPoint(kinesisClient, streamName, startingPoint));
+    return listShards(kinesisClient, streamName, shardFilter);
+  }
+
+  static ShardFilter buildShardFilterForStartingPoint(
+      KinesisClient kinesisClient, String streamName, StartingPoint startingPoint)
+      throws IOException, InterruptedException {
+    InitialPositionInStream position = startingPoint.getPosition();
+    switch (position) {
+      case LATEST:
+        return ShardFilter.builder().type(ShardFilterType.AT_LATEST).build();
+      case TRIM_HORIZON:
+        return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+      case AT_TIMESTAMP:
+        return buildShardFilterForTimestamp(
+            kinesisClient, streamName, startingPoint.getTimestamp());
+      default:
+        throw new IllegalArgumentException(
+            String.format("Unrecognized '%s' position to create shard filter with", position));
+    }
+  }
+
+  private static ShardFilter buildShardFilterForTimestamp(
+      KinesisClient kinesisClient, String streamName, Instant startingPointTimestamp)
+      throws IOException, InterruptedException {
+    StreamDescriptionSummary streamDescription = describeStreamSummary(kinesisClient, streamName);
+
+    Instant streamCreationTimestamp = TimeUtil.toJoda(streamDescription.streamCreationTimestamp());
+    if (streamCreationTimestamp.isAfter(startingPointTimestamp)) {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+
+    Duration retentionPeriod = Duration.standardHours(streamDescription.retentionPeriodHours());
+
+    Instant streamTrimHorizonTimestamp =
+        Instant.now()
+            .minus(retentionPeriod)
+            .plus(SPACING_FOR_TIMESTAMP_LIST_SHARDS_REQUEST_TO_NOT_EXCEED_TRIM_HORIZON);
+    if (startingPointTimestamp.isAfter(streamTrimHorizonTimestamp)) {
+      return ShardFilter.builder()
+          .type(ShardFilterType.AT_TIMESTAMP)
+          .timestamp(TimeUtil.toJava(startingPointTimestamp))
+          .build();
+    } else {
+      return ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build();
+    }
+  }
+
+  private static StreamDescriptionSummary describeStreamSummary(
+      KinesisClient kinesisClient, final String streamName)
+      throws IOException, InterruptedException {
+    // DescribeStreamSummary has limits that can be hit fairly easily if we are attempting
+    // to configure multiple KinesisIO inputs in the same account. Retry up to
+    // DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS times if we end up hitting that limit.
+    //
+    // Only pass the wrapped exception up once that limit is reached. Use FluentBackoff
+    // to implement the retry policy.
+    FluentBackoff retryBackoff =
+        FluentBackoff.DEFAULT
+            .withMaxRetries(DESCRIBE_STREAM_SUMMARY_MAX_ATTEMPTS)
+            .withInitialBackoff(DESCRIBE_STREAM_SUMMARY_INITIAL_BACKOFF);
+    BackOff backoff = retryBackoff.backoff();
+    Sleeper sleeper = Sleeper.DEFAULT;
+
+    DescribeStreamSummaryRequest request =
+        DescribeStreamSummaryRequest.builder().streamName(streamName).build();
+    while (true) {
+      try {
+        LOG.info("Executing request: {}", request);
+        return kinesisClient.describeStreamSummary(request).streamDescriptionSummary();
+      } catch (LimitExceededException exc) {
+        if (!BackOffUtils.next(sleeper, backoff)) {
+          throw exc;
+        }
+      }
+    }
+  }
+
+  static List<Shard> listShards(
+      KinesisClient kinesisClient, final String streamName, final ShardFilter shardFilter)
+      throws TransientKinesisException {
+    return wrapExceptions(
+        () -> {
+          ImmutableList.Builder<Shard> shardsBuilder = ImmutableList.builder();
+
+          String currentNextToken = null;
+          do {
+            ListShardsRequest.Builder reqBuilder =
+                ListShardsRequest.builder()
+                    .maxResults(LIST_SHARDS_MAX_RESULTS)
+                    .shardFilter(shardFilter);
+            if (currentNextToken != null) {
+              reqBuilder.nextToken(currentNextToken);
+            } else {
+              reqBuilder.streamName(streamName);
+            }
+
+            ListShardsRequest request = reqBuilder.build();
+            LOG.info("Executing request: {}", request);

Review Comment:
   Change to debug pls



##########
sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/Helpers.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.IntStream.range;
+import static org.joda.time.Duration.standardSeconds;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.argThat;
+import static org.mockito.Mockito.when;
+
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.mockito.ArgumentMatcher;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+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.ListShardsRequest;
+import software.amazon.awssdk.services.kinesis.model.ListShardsResponse;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+class Helpers {
+  static final int SHARD_EVENTS = 100;
+
+  static void mockShards(KinesisClient client, int count) {
+    IntFunction<Shard> shard = i -> Shard.builder().shardId(Integer.toString(i)).build();
+    List<Shard> shards = range(0, count).mapToObj(shard).collect(toList());
+    when(client.listShards(any(ListShardsRequest.class)))
+        .thenReturn(ListShardsResponse.builder().shards(shards).build());
+  }
+
+  static void mockShardIterators(KinesisClient client, List<List<Record>> data) {
+    for (int id = 0; id < data.size(); id++) {
+      when(client.getShardIterator(argThat(hasShardId(id))))
+          .thenReturn(GetShardIteratorResponse.builder().shardIterator(id + ":0").build());
+    }
+  }
+
+  static void mockRecords(KinesisClient client, List<List<Record>> data, int limit) {
+    BiFunction<List<Record>, String, GetRecordsResponse.Builder> resp =
+        (recs, it) ->
+            GetRecordsResponse.builder().millisBehindLatest(0L).records(recs).nextShardIterator(it);
+
+    for (int shard = 0; shard < data.size(); shard++) {
+      List<Record> records = data.get(shard);
+      for (int i = 0; i < records.size(); i += limit) {
+        int to = Math.max(i + limit, records.size());
+        String nextIt = (to == records.size()) ? "done" : shard + ":" + to;
+        when(client.getRecords(argThat(hasShardIterator(shard + ":" + i))))
+            .thenReturn(resp.apply(records.subList(i, to), nextIt).build());
+      }
+    }
+    when(client.getRecords(argThat(hasShardIterator("done"))))
+        .thenReturn(resp.apply(ImmutableList.of(), "done").build());
+  }
+
+  static List<List<Record>> testRecords(int shards, int events) {
+    final Instant now = DateTime.now().toInstant();
+    Function<Integer, List<Record>> dataStream =
+        shard -> range(0, events).mapToObj(off -> record(now, shard, off)).collect(toList());
+    return range(0, shards).boxed().map(dataStream).collect(toList());
+  }
+
+  static List<List<Record>> testAggregatedRecords(int shards, int events) {

Review Comment:
   Nitpick, `createAggregatedRecords`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org