hlteoh37 commented on code in PR #146:
URL: 
https://github.com/apache/flink-connector-aws/pull/146#discussion_r1670454462


##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/proxy/DynamoDbStreamsProxyTest.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.flink.connector.dynamodb.source.util.DynamoDbStreamsClientProvider.DescribeStreamItem;
+import 
org.apache.flink.connector.dynamodb.source.util.DynamoDbStreamsClientProvider.TestingDynamoDbStreamsClient;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.NullAndEmptySource;
+import org.junit.jupiter.params.provider.ValueSource;
+import software.amazon.awssdk.http.SdkHttpClient;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.dynamodb.model.DynamoDbStreamsRequest;
+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.Record;
+import software.amazon.awssdk.services.dynamodb.model.Shard;
+import software.amazon.awssdk.services.dynamodb.model.ShardIteratorType;
+import software.amazon.awssdk.services.dynamodb.model.StreamStatus;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.generateShardId;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static 
org.assertj.core.api.AssertionsForClassTypes.assertThatNoException;
+
+/** Tests to validate {@link DynamoDbStreamsProxy}. */
+public class DynamoDbStreamsProxyTest {
+    private static final SdkHttpClient HTTP_CLIENT = 
ApacheHttpClient.builder().build();
+
+    @ParameterizedTest
+    @NullAndEmptySource
+    @ValueSource(strings = {"shardId-000000000002"})
+    void testListShards(String lastSeenShardId) {
+        final String streamArn =
+                
"arn:aws:dynamodb:us-east-1:1231231230:table/test/stream/2024-01-01T00:00:00.826";
+        final List<Shard> expectedShards = getTestShards(0, 3);
+        List<DescribeStreamItem> describeStreamItems =
+                Stream.of(
+                                DescribeStreamItem.builder()
+                                        .validation(
+                                                
getDescribeStreamRequestValidation(
+                                                        streamArn, 
lastSeenShardId))
+                                        .nextToken(null)
+                                        .shards(expectedShards)
+                                        .streamStatus(StreamStatus.ENABLED)
+                                        .streamArn(streamArn)
+                                        .build(),
+                                DescribeStreamItem.builder()
+                                        .validation(
+                                                
getDescribeStreamRequestValidation(
+                                                        streamArn,
+                                                        expectedShards
+                                                                
.get(expectedShards.size() - 1)
+                                                                .shardId()))
+                                        .shards(Collections.EMPTY_LIST)
+                                        .streamStatus(StreamStatus.ENABLED)
+                                        .streamArn(streamArn)
+                                        .build())
+                        .collect(Collectors.toList());
+        TestingDynamoDbStreamsClient testingDynamoDbStreamsClient =
+                new TestingDynamoDbStreamsClient();
+        
testingDynamoDbStreamsClient.setDescribeStreamResponse(describeStreamItems);
+
+        DynamoDbStreamsProxy dynamoDbStreamsProxy =
+                new DynamoDbStreamsProxy(testingDynamoDbStreamsClient, 
HTTP_CLIENT);
+
+        assertThat(dynamoDbStreamsProxy.listShards(streamArn, lastSeenShardId))
+                .isEqualTo(expectedShards);
+    }
+
+    @Test
+    void testGetRecordsInitialReadFromTrimHorizon() {

Review Comment:
   can we test reading from latest too?



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/split/StartingPositionTest.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.split;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class StartingPositionTest {
+
+    @Test
+    void testEquals() {
+        StartingPosition startingPosition1 = StartingPosition.fromStart();
+        StartingPosition startingPosition2 = StartingPosition.fromStart();
+        StartingPosition startingPosition3 = StartingPosition.latest();
+        assertTrue(startingPosition1.equals(startingPosition2));
+        assertFalse(startingPosition1.equals(startingPosition3));
+    }

Review Comment:
   Why not use the equalsVerifier?



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/PollingDynamoDbStreamsShardSplitReaderTest.java:
##########
@@ -0,0 +1,229 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import 
org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
+import org.apache.flink.connector.dynamodb.source.proxy.StreamProxy;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplit;
+import 
org.apache.flink.connector.dynamodb.source.util.DynamoDbStreamsProxyProvider.TestDynamoDbStreamsProxy;
+import org.apache.flink.connector.dynamodb.source.util.TestUtil;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.DynamoDbStreamsProxyProvider.getTestStreamProxy;
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.generateShardId;
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestRecord;
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplit;
+import static org.assertj.core.api.Assertions.assertThat;
+import static 
org.assertj.core.api.AssertionsForClassTypes.assertThatNoException;
+
+class PollingDynamoDbStreamsShardSplitReaderTest {
+    @Test
+    void testNoAssignedSplitsHandledGracefully() throws Exception {
+        StreamProxy testStreamProxy = getTestStreamProxy();
+        PollingDynamoDbStreamsShardSplitReader splitReader =
+                new PollingDynamoDbStreamsShardSplitReader(testStreamProxy);
+
+        RecordsWithSplitIds<Record> retrievedRecords = splitReader.fetch();
+
+        assertThat(retrievedRecords.nextRecordFromSplit()).isNull();
+        assertThat(retrievedRecords.nextSplit()).isNull();
+        assertThat(retrievedRecords.finishedSplits()).isEmpty();
+    }
+
+    @Test
+    void testAssignedSplitHasNoRecordsHandledGracefully() throws Exception {
+        TestDynamoDbStreamsProxy testStreamProxy = getTestStreamProxy();
+        PollingDynamoDbStreamsShardSplitReader splitReader =
+                new PollingDynamoDbStreamsShardSplitReader(testStreamProxy);
+
+        // Given assigned split with no records
+        String shardId = generateShardId(1);
+        testStreamProxy.addShards(shardId);
+        splitReader.handleSplitsChanges(
+                new 
SplitsAddition<>(Collections.singletonList(getTestSplit(shardId))));
+
+        // When fetching records
+        RecordsWithSplitIds<Record> retrievedRecords = splitReader.fetch();
+
+        // Then retrieve no records
+        assertThat(retrievedRecords.nextRecordFromSplit()).isNull();
+        assertThat(retrievedRecords.nextSplit()).isNull();
+        assertThat(retrievedRecords.finishedSplits()).isEmpty();
+    }
+
+    @Test
+    void testSingleAssignedSplitAllConsumed() throws Exception {
+        TestDynamoDbStreamsProxy testStreamProxy = getTestStreamProxy();
+        PollingDynamoDbStreamsShardSplitReader splitReader =
+                new PollingDynamoDbStreamsShardSplitReader(testStreamProxy);
+
+        // Given assigned split with records
+        String shardId = generateShardId(1);
+        testStreamProxy.addShards(shardId);
+        List<Record> expectedRecords =
+                Stream.of(getTestRecord("data-1"), getTestRecord("data-2"), 
getTestRecord("data-3"))
+                        .collect(Collectors.toList());
+        testStreamProxy.addRecords(
+                TestUtil.STREAM_ARN, shardId, 
Collections.singletonList(expectedRecords.get(0)));
+        testStreamProxy.addRecords(
+                TestUtil.STREAM_ARN, shardId, 
Collections.singletonList(expectedRecords.get(1)));
+        testStreamProxy.addRecords(
+                TestUtil.STREAM_ARN, shardId, 
Collections.singletonList(expectedRecords.get(2)));
+        splitReader.handleSplitsChanges(
+                new 
SplitsAddition<>(Collections.singletonList(getTestSplit(shardId))));
+
+        // When fetching records
+        List<Record> records = new ArrayList<>();
+        for (int i = 0; i < expectedRecords.size(); i++) {
+            RecordsWithSplitIds<Record> retrievedRecords = splitReader.fetch();
+            records.addAll(readAllRecords(retrievedRecords));
+        }
+
+        
assertThat(records).containsExactlyInAnyOrderElementsOf(expectedRecords);
+    }
+
+    @Test
+    void testMultipleAssignedSplitsAllConsumed() throws Exception {
+        TestDynamoDbStreamsProxy testStreamProxy = getTestStreamProxy();

Review Comment:
   I think this is a miss (for KDS too) where we should be adding multiple 
splits



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(),
+                        startTime.plusSeconds(10).toEpochMilli(),
+                        startTime.plusSeconds(20).toEpochMilli());
+        assertThat(splitState.getNextStartingPosition())
+                .usingRecursiveComparison()
+                .isEqualTo(expectedStartingPosition);
+    }
+
+    @Test
+    void testEmitRecordBasedOnSequenceNumber() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(new String[] {"emit", "do-not-emit", 
"emit"});
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(new 
SequenceNumberBasedDeserializationSchema());
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        assertThat(emittedRecords.size()).isEqualTo(2);
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("740280c73a3df7842edab3548a1b08ad"));

Review Comment:
   Let's use `assertThat`



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/util/DynamoDbStreamsClientProvider.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.util;
+
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse;
+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.GetShardIteratorResponse;
+import software.amazon.awssdk.services.dynamodb.model.Shard;
+import software.amazon.awssdk.services.dynamodb.model.StreamDescription;
+import software.amazon.awssdk.services.dynamodb.model.StreamStatus;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import 
software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsServiceClientConfiguration;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Consumer;
+
+/** Provides {@link DynamoDbStreamsClient} with mocked DynamoDbStreams 
behavior. */
+public class DynamoDbStreamsClientProvider {
+
+    /**
+     * An implementation of the {@link DynamoDbStreamsClient} that allows 
control over DynamoDb
+     * Service responses.
+     */
+    public static class TestingDynamoDbStreamsClient implements 
DynamoDbStreamsClient {
+
+        private Deque<DescribeStreamItem> describeStreamItems = new 
ArrayDeque<>();
+        private Deque<String> shardIterators = new ArrayDeque<>();
+        private Consumer<GetShardIteratorRequest> getShardIteratorValidation;
+        private GetRecordsResponse getRecordsResponse;
+        private Consumer<GetRecordsRequest> getRecordsValidation;
+        private boolean closed = false;
+
+        @Override
+        public String serviceName() {
+            return "dynamodb";
+        }
+
+        @Override
+        public void close() {
+            closed = true;
+        }
+
+        public boolean isClosed() {
+            return closed;
+        }
+
+        public void setNextShardIterator(String shardIterator) {
+            this.shardIterators.add(shardIterator);
+        }
+
+        public void 
setShardIteratorValidation(Consumer<GetShardIteratorRequest> validation) {
+            this.getShardIteratorValidation = validation;
+        }
+
+        @Override
+        public GetShardIteratorResponse getShardIterator(
+                GetShardIteratorRequest getShardIteratorRequest)
+                throws AwsServiceException, SdkClientException {
+            getShardIteratorValidation.accept(getShardIteratorRequest);
+            return 
GetShardIteratorResponse.builder().shardIterator(shardIterators.poll()).build();
+        }
+
+        public void setDescribeStreamResponse(List<DescribeStreamItem> items) {
+            describeStreamItems.addAll(items);
+        }
+
+        @Override
+        public DescribeStreamResponse describeStream(DescribeStreamRequest 
describeStreamRequest)
+                throws AwsServiceException, SdkClientException {
+
+            DescribeStreamItem item = describeStreamItems.pop();
+
+            item.validation.accept(describeStreamRequest);
+            return DescribeStreamResponse.builder()
+                    .streamDescription(
+                            StreamDescription.builder()
+                                    .shards(item.shards)
+                                    .streamStatus(item.streamStatus)
+                                    .streamArn(item.streamArn)
+                                    .build())
+                    .build();
+        }
+
+        public void setGetRecordsResponse(GetRecordsResponse 
getRecordsResponse) {
+            this.getRecordsResponse = getRecordsResponse;
+        }
+
+        public void setGetRecordsValidation(Consumer<GetRecordsRequest> 
validation) {
+            this.getRecordsValidation = validation;
+        }
+
+        @Override
+        public GetRecordsResponse getRecords(GetRecordsRequest 
getRecordsRequest)
+                throws AwsServiceException, SdkClientException {
+            getRecordsValidation.accept(getRecordsRequest);
+            return getRecordsResponse;
+        }
+
+        @Override
+        public DynamoDbStreamsServiceClientConfiguration 
serviceClientConfiguration() {
+            // This is not used
+            return null;
+        }
+    }
+
+    /** Data class to provide a mocked response to ListShards() calls. */
+    public static class DescribeStreamItem {
+        private final Consumer<DescribeStreamRequest> validation;
+        private final List<Shard> shards;
+        private final String nextToken;
+        private final StreamStatus streamStatus;
+        private final String streamArn;
+
+        private DescribeStreamItem(
+                Consumer<DescribeStreamRequest> validation,
+                List<Shard> shards,
+                String nextToken,
+                StreamStatus streamStatus,
+                String streamArn) {
+            this.validation = validation;
+            this.shards = shards;
+            this.nextToken = nextToken;
+            this.streamStatus = streamStatus;
+            this.streamArn = streamArn;
+        }
+
+        public static Builder builder() {
+            return new Builder();
+        }
+
+        /** Builder for {@link DescribeStreamItem}. */
+        public static class Builder {
+            private Consumer<DescribeStreamRequest> validation;
+            private List<Shard> shards;
+            private String nextToken;

Review Comment:
   Why can't we use the SDK class?



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(),
+                        startTime.plusSeconds(10).toEpochMilli(),
+                        startTime.plusSeconds(20).toEpochMilli());
+        assertThat(splitState.getNextStartingPosition())
+                .usingRecursiveComparison()
+                .isEqualTo(expectedStartingPosition);
+    }
+
+    @Test
+    void testEmitRecordBasedOnSequenceNumber() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(new String[] {"emit", "do-not-emit", 
"emit"});
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(new 
SequenceNumberBasedDeserializationSchema());
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }

Review Comment:
   What does this actually test? IS this testing the 
`SequenceNumberBasedDeserializationSchema`? If so, that's a class that only 
specified in test. Can we consider removing?



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));

Review Comment:
   can we use `assertThat` instead? The failure message is better.



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(),
+                        startTime.plusSeconds(10).toEpochMilli(),
+                        startTime.plusSeconds(20).toEpochMilli());
+        assertThat(splitState.getNextStartingPosition())
+                .usingRecursiveComparison()
+                .isEqualTo(expectedStartingPosition);
+    }
+
+    @Test
+    void testEmitRecordBasedOnSequenceNumber() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(new String[] {"emit", "do-not-emit", 
"emit"});
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(new 
SequenceNumberBasedDeserializationSchema());
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        assertThat(emittedRecords.size()).isEqualTo(2);
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(), 
startTime.plusSeconds(20).toEpochMilli());
+    }
+
+    @Test
+    void testEmitRecordWithMetadata() throws Exception {
+        final Instant startTime = Instant.now();

Review Comment:
   What does this test? Might be better if we instead assert that the 
deserialization schema is passed the metadata, sequence number, etc? That would 
enforce the API interface better :) 



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/util/DynamoDbStreamsClientProvider.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.util;
+
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse;
+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.GetShardIteratorResponse;
+import software.amazon.awssdk.services.dynamodb.model.Shard;
+import software.amazon.awssdk.services.dynamodb.model.StreamDescription;
+import software.amazon.awssdk.services.dynamodb.model.StreamStatus;
+import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient;
+import 
software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsServiceClientConfiguration;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.List;
+import java.util.function.Consumer;
+
+/** Provides {@link DynamoDbStreamsClient} with mocked DynamoDbStreams 
behavior. */
+public class DynamoDbStreamsClientProvider {
+
+    /**
+     * An implementation of the {@link DynamoDbStreamsClient} that allows 
control over DynamoDb
+     * Service responses.
+     */
+    public static class TestingDynamoDbStreamsClient implements 
DynamoDbStreamsClient {
+
+        private Deque<DescribeStreamItem> describeStreamItems = new 
ArrayDeque<>();
+        private Deque<String> shardIterators = new ArrayDeque<>();
+        private Consumer<GetShardIteratorRequest> getShardIteratorValidation;
+        private GetRecordsResponse getRecordsResponse;
+        private Consumer<GetRecordsRequest> getRecordsValidation;
+        private boolean closed = false;
+
+        @Override
+        public String serviceName() {
+            return "dynamodb";
+        }
+
+        @Override
+        public void close() {
+            closed = true;
+        }
+
+        public boolean isClosed() {
+            return closed;
+        }
+
+        public void setNextShardIterator(String shardIterator) {
+            this.shardIterators.add(shardIterator);
+        }
+
+        public void 
setShardIteratorValidation(Consumer<GetShardIteratorRequest> validation) {
+            this.getShardIteratorValidation = validation;
+        }
+
+        @Override
+        public GetShardIteratorResponse getShardIterator(
+                GetShardIteratorRequest getShardIteratorRequest)
+                throws AwsServiceException, SdkClientException {
+            getShardIteratorValidation.accept(getShardIteratorRequest);
+            return 
GetShardIteratorResponse.builder().shardIterator(shardIterators.poll()).build();
+        }
+
+        public void setDescribeStreamResponse(List<DescribeStreamItem> items) {
+            describeStreamItems.addAll(items);
+        }

Review Comment:
   Do we want `describeStreamValidation`?



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(),
+                        startTime.plusSeconds(10).toEpochMilli(),
+                        startTime.plusSeconds(20).toEpochMilli());
+        assertThat(splitState.getNextStartingPosition())
+                .usingRecursiveComparison()
+                .isEqualTo(expectedStartingPosition);
+    }
+
+    @Test
+    void testEmitRecordBasedOnSequenceNumber() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(new String[] {"emit", "do-not-emit", 
"emit"});
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(new 
SequenceNumberBasedDeserializationSchema());
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        assertThat(emittedRecords.size()).isEqualTo(2);
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("740280c73a3df7842edab3548a1b08ad"));
+        assertThat(output.getEmittedTimestamps())
+                .containsExactly(
+                        startTime.toEpochMilli(), 
startTime.plusSeconds(20).toEpochMilli());
+    }
+
+    @Test
+    void testEmitRecordWithMetadata() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        new AssertRecordMetadataDeserializationSchema(
+                                splitState.getStreamArn(), 
splitState.getShardId()));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));

Review Comment:
   Let's use assertThat.



##########
flink-connector-aws/flink-connector-dynamodb/src/test/java/org/apache/flink/connector/dynamodb/source/reader/DynamoDbStreamsRecordEmitterTest.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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.reader;
+
+import org.apache.flink.api.common.eventtime.Watermark;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.SourceOutput;
+import 
org.apache.flink.connector.dynamodb.source.serialization.DynamoDbStreamsDeserializationSchema;
+import 
org.apache.flink.connector.dynamodb.source.serialization.RecordObjectMapper;
+import 
org.apache.flink.connector.dynamodb.source.split.DynamoDbStreamsShardSplitState;
+import org.apache.flink.connector.dynamodb.source.split.StartingPosition;
+import org.apache.flink.util.Collector;
+
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.Record;
+import software.amazon.awssdk.services.dynamodb.model.StreamRecord;
+import software.amazon.awssdk.utils.ImmutableMap;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.connector.dynamodb.source.util.TestUtil.getTestSplitState;
+import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class DynamoDbStreamsRecordEmitterTest {
+
+    private static final SimpleStringSchema STRING_SCHEMA = new 
SimpleStringSchema();
+
+    private List<Record> getDynamoDbStreamsRecords(String[] sequenceNumbers) {
+        final Instant startTime = Instant.now();
+        return Stream.of(
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                
.approximateCreationDateTime(startTime)
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[0])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("4b25bd0da9a181a155114127e4837252")
+                                .eventName("INSERT")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(10))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .newImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("50")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[1])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("e2fd9c34eff2d779b297b26f5fef4206")
+                                .eventName("MODIFY")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build(),
+                        Record.builder()
+                                .dynamodb(
+                                        StreamRecord.builder()
+                                                .approximateCreationDateTime(
+                                                        
startTime.plusSeconds(20))
+                                                .keys(
+                                                        ImmutableMap.of(
+                                                                "ForumName",
+                                                                
AttributeValue.builder()
+                                                                        
.s("DynamoDB")
+                                                                        
.build()))
+                                                .oldImage(
+                                                        ImmutableMap.of(
+                                                                "quantity",
+                                                                
AttributeValue.builder()
+                                                                        
.n("20")
+                                                                        
.build()))
+                                                
.sequenceNumber(sequenceNumbers[2])
+                                                .sizeBytes(41L)
+                                                
.streamViewType("NEW_AND_OLD_IMAGES")
+                                                .build())
+                                .eventID("740280c73a3df7842edab3548a1b08ad")
+                                .eventName("REMOVE")
+                                .eventSource("aws:dynamodb")
+                                .eventVersion("1.0")
+                                .build())
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    void testEmitRecord() throws Exception {
+        final Instant startTime = Instant.now();
+        List<Record> inputRecords =
+                getDynamoDbStreamsRecords(
+                        new String[] {
+                            "300000000000000499659",
+                            "400000000000000499660",
+                            "500000000000000499661"
+                        });
+        final StartingPosition expectedStartingPosition =
+                
StartingPosition.continueFromSequenceNumber("500000000000000499661");
+        final CapturingSourceOutput<String> output = new 
CapturingSourceOutput<>();
+        final DynamoDbStreamsShardSplitState splitState = getTestSplitState();
+
+        DynamoDbStreamsRecordEmitter<String> emitter =
+                new DynamoDbStreamsRecordEmitter<>(
+                        
DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA));
+        for (Record record : inputRecords) {
+            emitter.emitRecord(record, output, splitState);
+        }
+
+        List<String> emittedRecords = output.emittedRecords;
+        
assertTrue(emittedRecords.get(0).contains("4b25bd0da9a181a155114127e4837252"));
+        
assertTrue(emittedRecords.get(1).contains("e2fd9c34eff2d779b297b26f5fef4206"));
+        
assertTrue(emittedRecords.get(2).contains("740280c73a3df7842edab3548a1b08ad"));

Review Comment:
   It might be nicer to use the 
`DynamoDbStreamsDeserializationSchema.of(STRING_SCHEMA)` and validate the 
output is exactly equal, rather than just looking for eventID!
   



-- 
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