dannycranmer commented on code in PR #49:
URL: 
https://github.com/apache/flink-connector-aws/pull/49#discussion_r1192958496


##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/proxy/StreamProxy.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.flink.connector.kinesis.source.proxy;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+
+/** Interface for a StreamProxy to interact with Streams service in a given 
region. */
+@Internal
+public interface StreamProxy {

Review Comment:
   You should add a `close()` method or make `Closable` 



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/KinesisStreamsSource.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.flink.connector.kinesis.source;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.aws.util.AWSClientUtil;
+import org.apache.flink.connector.aws.util.AWSGeneralUtil;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import 
org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
+import 
org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
+import org.apache.flink.connector.kinesis.sink.KinesisStreamsConfigConstants;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisStreamsSourceEnumerator;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisStreamsSourceEnumeratorState;
+import 
org.apache.flink.connector.kinesis.source.enumerator.KinesisStreamsSourceEnumeratorStateSerializer;
+import org.apache.flink.connector.kinesis.source.proxy.KinesisStreamProxy;
+import 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsRecordEmitter;
+import 
org.apache.flink.connector.kinesis.source.reader.KinesisStreamsSourceReader;
+import 
org.apache.flink.connector.kinesis.source.reader.PollingKinesisShardSplitReader;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import 
org.apache.flink.connector.kinesis.source.split.KinesisShardSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import software.amazon.awssdk.http.SdkHttpClient;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
+import software.amazon.awssdk.services.kinesis.KinesisClient;
+import software.amazon.awssdk.services.kinesis.model.Record;
+import software.amazon.awssdk.utils.AttributeMap;
+
+import java.util.Properties;
+import java.util.function.Supplier;
+
+/**
+ * The {@link KinesisStreamsSource} is an exactly-once parallel streaming data 
source that
+ * subscribes to a single AWS Kinesis data stream. It is able to handle 
resharding of streams, and
+ * stores its current progress in Flink checkpoints. The source will read in 
data from the Kinesis
+ * Data stream, deserialize it using the provided {@link 
DeserializationSchema}, and emit the record
+ * into the Flink job graph.
+ *
+ * <p>Exactly-once semantics. To leverage Flink's checkpointing mechanics for 
exactly-once stream
+ * processing, the Kinesis Source is implemented with the AWS Java SDK, 
instead of the officially
+ * recommended AWS Kinesis Client Library. The source will store its current 
progress in Flink
+ * checkpoint/savepoint, and will pick up from where it left off upon restore 
from the
+ * checkpoint/savepoint.
+ *
+ * <p>Initial starting points. The Kinesis Streams Source supports reads 
starting from TRIM_HORIZON,
+ * LATEST, and AT_TIMESTAMP.
+ *
+ * @param <T> the data type emitted by the source
+ */
+@Experimental
+public class KinesisStreamsSource<T>
+        implements Source<T, KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final DeserializationSchema<T> deserializationSchema;
+
+    public KinesisStreamsSource(
+            String streamArn,
+            Properties consumerConfig,
+            DeserializationSchema<T> deserializationSchema) {
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.deserializationSchema = deserializationSchema;
+    }
+
+    @Override
+    public Boundedness getBoundedness() {
+        return Boundedness.CONTINUOUS_UNBOUNDED;
+    }
+
+    @Override
+    public SourceReader<T, KinesisShardSplit> createReader(SourceReaderContext 
readerContext)
+            throws Exception {
+        FutureCompletingBlockingQueue<RecordsWithSplitIds<Record>> 
elementsQueue =
+                new FutureCompletingBlockingQueue<>();
+        KinesisStreamProxy kinesisStreamProxy = 
createKinesisStreamProxy(consumerConfig);
+        Supplier<PollingKinesisShardSplitReader> splitReaderSupplier =
+                () -> new PollingKinesisShardSplitReader(kinesisStreamProxy);
+        KinesisStreamsRecordEmitter<T> recordEmitter =
+                new KinesisStreamsRecordEmitter<>(deserializationSchema);
+
+        return new KinesisStreamsSourceReader<>(
+                elementsQueue,
+                new SingleThreadFetcherManager<>(elementsQueue, 
splitReaderSupplier::get),
+                recordEmitter,
+                convertPropertiesToConfiguration(consumerConfig),
+                readerContext);
+    }
+
+    @Override
+    public SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> createEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> enumContext) throws 
Exception {
+        return restoreEnumerator(enumContext, null);
+    }
+
+    @Override
+    public SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState>
+            restoreEnumerator(
+                    SplitEnumeratorContext<KinesisShardSplit> enumContext,
+                    KinesisStreamsSourceEnumeratorState checkpoint)
+                    throws Exception {
+        return new KinesisStreamsSourceEnumerator(
+                enumContext,
+                streamArn,
+                consumerConfig,
+                createKinesisStreamProxy(consumerConfig),
+                checkpoint);
+    }
+
+    @Override
+    public SimpleVersionedSerializer<KinesisShardSplit> getSplitSerializer() {
+        return new KinesisShardSplitSerializer();
+    }
+
+    @Override
+    public SimpleVersionedSerializer<KinesisStreamsSourceEnumeratorState>
+            getEnumeratorCheckpointSerializer() {
+        return new KinesisStreamsSourceEnumeratorStateSerializer(new 
KinesisShardSplitSerializer());
+    }
+
+    private Configuration convertPropertiesToConfiguration(Properties props) {
+        Configuration config = new Configuration();
+        props.stringPropertyNames().forEach(key -> config.setString(key, 
props.getProperty(key)));
+        return config;
+    }
+
+    private KinesisStreamProxy createKinesisStreamProxy(Properties 
consumerConfig) {
+        SdkHttpClient httpClient =
+                AWSGeneralUtil.createSyncHttpClient(
+                        AttributeMap.builder().build(), 
ApacheHttpClient.builder());
+
+        AWSGeneralUtil.validateAwsCredentials(consumerConfig);
+        KinesisClient kinesisClient =
+                AWSClientUtil.createAwsSyncClient(

Review Comment:
   These clients need to be closed, both the HTTP client and Kinesis client



##########
flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/enumerator/KinesisStreamsSourceEnumerator.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.flink.connector.kinesis.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.ReaderInfo;
+import org.apache.flink.api.connector.source.SourceEvent;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.InitialPosition;
+import 
org.apache.flink.connector.kinesis.source.enumerator.assigner.ShardAssignerFactory;
+import 
org.apache.flink.connector.kinesis.source.exception.KinesisStreamsSourceException;
+import org.apache.flink.connector.kinesis.source.model.CompletedShardsEvent;
+import org.apache.flink.connector.kinesis.source.proxy.StreamProxy;
+import org.apache.flink.connector.kinesis.source.split.KinesisShardSplit;
+import org.apache.flink.connector.kinesis.source.split.StartingPosition;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.model.Shard;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.DEFAULT_STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.SHARD_DISCOVERY_INTERVAL_MILLIS;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigConstants.STREAM_INITIAL_POSITION;
+import static 
org.apache.flink.connector.kinesis.source.config.SourceConfigUtil.parseStreamTimestampStartingPosition;
+
+/**
+ * This class is used to discover and assign Kinesis splits to subtasks on the 
Flink cluster. This
+ * runs on the JobManager.
+ */
+@Internal
+public class KinesisStreamsSourceEnumerator
+        implements SplitEnumerator<KinesisShardSplit, 
KinesisStreamsSourceEnumeratorState> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KinesisStreamsSourceEnumerator.class);
+
+    private final SplitEnumeratorContext<KinesisShardSplit> context;
+    private final String streamArn;
+    private final Properties consumerConfig;
+    private final StreamProxy streamProxy;
+    private final KinesisShardAssigner shardAssigner;
+    private final KinesisShardAssigner.Context shardAssignerContext;
+
+    private final Map<Integer, Set<KinesisShardSplit>> splitAssignment = new 
HashMap<>();
+    private final Set<String> assignedSplitIds = new HashSet<>();
+    private final Set<KinesisShardSplit> unassignedSplits;
+    private final Set<String> completedSplitIds;
+
+    private String lastSeenShardId;
+
+    public KinesisStreamsSourceEnumerator(
+            SplitEnumeratorContext<KinesisShardSplit> context,
+            String streamArn,
+            Properties consumerConfig,
+            StreamProxy streamProxy,
+            KinesisStreamsSourceEnumeratorState state) {
+        this.context = context;
+        this.streamArn = streamArn;
+        this.consumerConfig = consumerConfig;
+        this.streamProxy = streamProxy;
+        this.shardAssigner = ShardAssignerFactory.uniformShardAssigner();
+        this.shardAssignerContext = new ShardAssignerContext(splitAssignment, 
context);
+        if (state == null) {
+            this.completedSplitIds = new HashSet<>();
+            this.lastSeenShardId = null;
+            this.unassignedSplits = new HashSet<>();
+        } else {
+            this.completedSplitIds = state.getCompletedSplitIds();
+            this.lastSeenShardId = state.getLastSeenShardId();
+            this.unassignedSplits = state.getUnassignedSplits();
+        }
+    }
+
+    @Override
+    public void start() {
+        if (lastSeenShardId == null) {
+            context.callAsync(this::initialDiscoverSplits, this::assignSplits);
+        }
+
+        final long shardDiscoveryInterval =
+                Long.parseLong(
+                        consumerConfig.getProperty(
+                                SHARD_DISCOVERY_INTERVAL_MILLIS,
+                                
String.valueOf(DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
+        context.callAsync(
+                this::periodicallyDiscoverSplits,
+                this::assignSplits,
+                shardDiscoveryInterval,
+                shardDiscoveryInterval);
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String 
requesterHostname) {
+        // Do nothing, since we assign splits eagerly
+    }
+
+    @Override
+    public void addSplitsBack(List<KinesisShardSplit> splits, int subtaskId) {
+        if (!splitAssignment.containsKey(subtaskId)) {
+            LOG.warn(
+                    "Unable to add splits back for subtask {} since it is not 
assigned any splits. Splits: {}",
+                    subtaskId,
+                    splits);
+            return;
+        }
+
+        for (KinesisShardSplit split : splits) {
+            splitAssignment.get(subtaskId).remove(split);
+            assignedSplitIds.remove(split.splitId());
+            unassignedSplits.add(split);
+        }
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        splitAssignment.putIfAbsent(subtaskId, new HashSet<>());
+    }
+
+    @Override
+    public KinesisStreamsSourceEnumeratorState snapshotState(long 
checkpointId) throws Exception {
+        return new KinesisStreamsSourceEnumeratorState(
+                completedSplitIds, unassignedSplits, lastSeenShardId);
+    }
+
+    @Override
+    public void close() throws IOException {}

Review Comment:
   @You need to close the Kinesis and HTTP client here; delegate to 
`StreamProxy::close`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to