hlteoh37 commented on code in PR #146: URL: https://github.com/apache/flink-connector-aws/pull/146#discussion_r1670132296
########## flink-connector-aws/flink-connector-dynamodb/src/main/java/org/apache/flink/connector/dynamodb/source/config/DynamodbStreamsSourceConfigConstants.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.dynamodb.source.config; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** Constants to be used with the DynamodbStreamsSource. */ +@Experimental +public class DynamodbStreamsSourceConfigConstants { + /** Marks the initial position to use when reading from the Dynamodb stream. */ + public enum InitialPosition { + LATEST, + TRIM_HORIZON, + AT_TIMESTAMP Review Comment: Can we remove this, since we don't support it? ########## flink-connector-aws/flink-connector-dynamodb/src/main/java/org/apache/flink/connector/dynamodb/source/proxy/DynamoDbStreamsProxy.java: ########## @@ -0,0 +1,184 @@ +/* + * 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.dynamodb.source.proxy; + +import org.apache.flink.connector.dynamodb.source.split.StartingPosition; + +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse; +import software.amazon.awssdk.services.dynamodb.model.ExpiredIteratorException; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsRequest; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsResponse; +import software.amazon.awssdk.services.dynamodb.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.Shard; +import software.amazon.awssdk.services.dynamodb.model.StreamStatus; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** Implementation of the {@link StreamProxy} for DynamoDB streams. */ +public class DynamoDbStreamsProxy implements StreamProxy { + + private final DynamoDbStreamsClient dynamoDbStreamsClient; + private final SdkHttpClient httpClient; + private final Map<String, String> shardIdToIteratorStore; + + private static final FullJitterBackoff BACKOFF = new FullJitterBackoff(); + + private static final Logger LOG = LoggerFactory.getLogger(DynamoDbStreamsProxy.class); + + public DynamoDbStreamsProxy(DynamoDbStreamsClient dynamoDbStreamsClient, SdkHttpClient httpClient) { + this.dynamoDbStreamsClient = dynamoDbStreamsClient; + this.httpClient = httpClient; + this.shardIdToIteratorStore = new ConcurrentHashMap<>(); + } + + @Override + public List<Shard> listShards(String streamArn, @Nullable String lastSeenShardId) { + return this.getShardsOfStream(streamArn, lastSeenShardId); + } + + @Override + public GetRecordsResponse getRecords(String streamArn, String shardId, StartingPosition startingPosition) { + String shardIterator = + shardIdToIteratorStore.computeIfAbsent( + shardId, (s) -> getShardIterator(streamArn, s, startingPosition)); + + if (shardIterator == null) { + return null; + } Review Comment: I would prefer if we returned an empty `GetRecordsResponse` instead of `null`. For example, later down in line 85 + 86 we didn't do a null check for `GetRecordsReponse`. ########## flink-connector-aws/flink-connector-dynamodb/src/main/java/org/apache/flink/connector/dynamodb/source/config/DynamodbStreamsSourceConfigConstants.java: ########## @@ -0,0 +1,67 @@ +/* + * 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.dynamodb.source.config; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** Constants to be used with the DynamodbStreamsSource. */ +@Experimental +public class DynamodbStreamsSourceConfigConstants { + /** Marks the initial position to use when reading from the Dynamodb stream. */ + public enum InitialPosition { + LATEST, + TRIM_HORIZON, + AT_TIMESTAMP + } + + public static final ConfigOption<InitialPosition> STREAM_INITIAL_POSITION = + ConfigOptions.key("flink.stream.initpos") + .enumType(InitialPosition.class) + .defaultValue(InitialPosition.LATEST) + .withDescription("The initial position to start reading Dynamodb streams."); + + public static final ConfigOption<String> STREAM_INITIAL_TIMESTAMP = + ConfigOptions.key("flink.stream.initpos.timestamp") + .stringType() + .noDefaultValue() + .withDescription( + "The initial timestamp at which to start reading from the Dynamodb stream. This is used when AT_TIMESTAMP is configured for the STREAM_INITIAL_POSITION."); + + public static final ConfigOption<String> STREAM_TIMESTAMP_DATE_FORMAT = + ConfigOptions.key("flink.stream.initpos.timestamp.format") + .stringType() + .defaultValue("yyyy-MM-dd'T'HH:mm:ss.SSSXXX") + .withDescription( + "The date format used to parse the initial timestamp at which to start reading from the Dynamodb stream. This is used when AT_TIMESTAMP is configured for the STREAM_INITIAL_POSITION."); Review Comment: If we don't support `AT_TIMESTAMP`, let's remove this. -- 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]
