jsun98 commented on a change in pull request #6431: Add Kinesis Indexing 
Service to core Druid
URL: https://github.com/apache/incubator-druid/pull/6431#discussion_r243099412
 
 

 ##########
 File path: 
extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java
 ##########
 @@ -0,0 +1,705 @@
+/*
+ * 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.druid.indexing.kinesis;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.services.kinesis.AmazonKinesis;
+import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder;
+import com.amazonaws.services.kinesis.model.GetRecordsRequest;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import 
com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
+import com.amazonaws.services.kinesis.model.Record;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import 
com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClientBuilder;
+import com.amazonaws.util.AwsHostNameUtils;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Queues;
+import org.apache.druid.common.aws.AWSCredentialsUtils;
+import org.apache.druid.indexing.kinesis.aws.ConstructibleAWSCredentialsConfig;
+import 
org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
+import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
+import org.apache.druid.indexing.seekablestream.common.StreamPartition;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.emitter.EmittingLogger;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * This class implements a local buffer for storing fetched Kinesis records. 
Fetching is done
+ * in background threads.
+ */
+public class KinesisRecordSupplier implements RecordSupplier<String, String>
+{
+  private static final EmittingLogger log = new 
EmittingLogger(KinesisRecordSupplier.class);
+  private static final long PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS = 3000;
+  private static final long EXCEPTION_RETRY_DELAY_MS = 10000;
+
+  private class PartitionResource
+  {
+    private final StreamPartition<String> streamPartition;
+    private final Object startLock = new Object();
+
+    private volatile String shardIterator;
+    private volatile boolean started;
+    private volatile boolean stopRequested;
+
+    PartitionResource(
+        StreamPartition<String> streamPartition
+    )
+    {
+      this.streamPartition = streamPartition;
+    }
+
+    void start()
+    {
+      synchronized (startLock) {
+        if (started) {
+          return;
+        }
+
+        log.info(
+            "Starting scheduled fetch runnable for stream[%s] partition[%s]",
+            streamPartition.getStream(),
+            streamPartition.getPartitionId()
+        );
+
+        stopRequested = false;
+        started = true;
+
+        rescheduleRunnable(fetchDelayMillis);
+      }
+    }
+
+    public void stop()
+    {
+      log.info(
+          "Stopping scheduled fetch runnable for stream[%s] partition[%s]",
+          streamPartition.getStream(),
+          streamPartition.getPartitionId()
+      );
+      stopRequested = true;
+    }
+
+
+    private Runnable getRecordRunnable()
+    {
+      return () -> {
+
+        if (stopRequested) {
+          started = false;
+          stopRequested = false;
+
+          log.info("Worker for partition[%s] has been stopped", 
streamPartition.getPartitionId());
+          return;
+        }
+
+
+        try {
+
+          if (shardIterator == null) {
+            log.info("shardIterator[%s] has been closed and has no more 
records", streamPartition.getPartitionId());
+
+            // add an end-of-shard marker so caller knows this shard is closed
+            OrderedPartitionableRecord<String, String> endOfShardRecord = new 
OrderedPartitionableRecord<>(
+                streamPartition.getStream(),
+                streamPartition.getPartitionId(),
+                OrderedPartitionableRecord.END_OF_SHARD_MARKER,
+                null
+            );
+
+            if (!records.offer(endOfShardRecord, recordBufferOfferTimeout, 
TimeUnit.MILLISECONDS)) {
+              log.warn("OrderedPartitionableRecord buffer full, retrying in 
[%,dms]", recordBufferFullWait);
+              rescheduleRunnable(recordBufferFullWait);
+            }
+
+            return;
+          }
+
+          GetRecordsResult recordsResult = kinesis.getRecords(new 
GetRecordsRequest().withShardIterator(
+              shardIterator).withLimit(recordsPerFetch));
+
+          // list will come back empty if there are no records
+          for (Record kinesisRecord : recordsResult.getRecords()) {
+
+            final List<byte[]> data;
+
+            if (deaggregate) {
+              if (deaggregateHandle == null || getDataHandle == null) {
+                throw new ISE("deaggregateHandle or getDataHandle is null!");
+              }
+
+              data = new ArrayList<>();
+
+              final List userRecords = (List) deaggregateHandle.invokeExact(
+                  Collections.singletonList(kinesisRecord)
+              );
+
+              for (Object userRecord : userRecords) {
+                data.add(toByteArray((ByteBuffer) 
getDataHandle.invoke(userRecord)));
+              }
+            } else {
+              data = 
Collections.singletonList(toByteArray(kinesisRecord.getData()));
+            }
+
+            final OrderedPartitionableRecord<String, String> record = new 
OrderedPartitionableRecord<>(
+                streamPartition.getStream(),
+                streamPartition.getPartitionId(),
+                kinesisRecord.getSequenceNumber(),
+                data
+            );
+
+
+            if (log.isTraceEnabled()) {
+              log.trace(
+                  "Stream[%s] / partition[%s] / sequenceNum[%s] / 
bufferRemainingCapacity[%d]: %s",
+                  record.getStream(),
+                  record.getPartitionId(),
+                  record.getSequenceNumber(),
+                  records.remainingCapacity(),
+                  
record.getData().stream().map(StringUtils::fromUtf8).collect(Collectors.toList())
+              );
+            }
+
+            // If the buffer was full and we weren't able to add the message, 
grab a new stream iterator starting
+            // from this message and back off for a bit to let the buffer 
drain before retrying.
+            if (!records.offer(record, recordBufferOfferTimeout, 
TimeUnit.MILLISECONDS)) {
+              log.warn(
+                  "OrderedPartitionableRecord buffer full, storing iterator 
and retrying in [%,dms]",
+                  recordBufferFullWait
+              );
+
+              shardIterator = kinesis.getShardIterator(
+                  record.getStream(),
+                  record.getPartitionId(),
+                  ShardIteratorType.AT_SEQUENCE_NUMBER.toString(),
+                  record.getSequenceNumber()
+              ).getShardIterator();
+
+              rescheduleRunnable(recordBufferFullWait);
+              return;
+            }
+          }
+
+          shardIterator = recordsResult.getNextShardIterator(); // will be 
null if the shard has been closed
+
+          rescheduleRunnable(fetchDelayMillis);
+        }
+        catch (ProvisionedThroughputExceededException e) {
+          long retryMs = Math.max(PROVISIONED_THROUGHPUT_EXCEEDED_BACKOFF_MS, 
fetchDelayMillis);
+          rescheduleRunnable(retryMs);
+        }
+        catch (Throwable e) {
 
 Review comment:
   I added comments and more catch clauses to improve error handling. However I 
think breaking the try clause into smaller ones would introduce duplicate error 
handling code such as `InterruptedException`, 
`ProvisionedThroughputExceededException` and `ExpiredIteratorException`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to