Skip to content

Commit 68adbcc

Browse files
elphastoridannycranmer
authored andcommitted
[FLINK-26890][Connector/Kinesis] Handle invalid shards in DynamoDB streams consumer
1 parent 9e1ff64 commit 68adbcc

File tree

3 files changed

+460
-0
lines changed

3 files changed

+460
-0
lines changed

flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient;
2828
import com.amazonaws.services.kinesis.AmazonKinesis;
2929
import com.amazonaws.services.kinesis.model.DescribeStreamResult;
30+
import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
3031
import com.amazonaws.services.kinesis.model.Shard;
3132
import org.slf4j.Logger;
3233
import org.slf4j.LoggerFactory;
@@ -110,6 +111,22 @@ public GetShardListResult getShardList(Map<String, String> streamNamesWithLastSe
110111
return result;
111112
}
112113

114+
@Override
115+
public String getShardIterator(
116+
StreamShardHandle shard, String shardIteratorType, @Nullable Object startingMarker)
117+
throws InterruptedException {
118+
try {
119+
return super.getShardIterator(shard, shardIteratorType, startingMarker);
120+
} catch (ResourceNotFoundException re) {
121+
LOG.info(
122+
"Received ResourceNotFoundException. "
123+
+ "Shard {} of stream {} is no longer valid, marking it as complete.",
124+
shard.getShard().getShardId(),
125+
shard.getStreamName());
126+
return null;
127+
}
128+
}
129+
113130
private List<StreamShardHandle> getShardsOfStream(
114131
String streamName, @Nullable String lastSeenShardId) throws InterruptedException {
115132
List<StreamShardHandle> shardsOfStream = new ArrayList<>();
Lines changed: 82 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,82 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.flink.streaming.connectors.kinesis.proxy;
19+
20+
import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
21+
import org.apache.flink.streaming.connectors.kinesis.testutils.FakeKinesisClientFactory;
22+
import org.apache.flink.streaming.connectors.kinesis.testutils.TestUtils;
23+
24+
import com.amazonaws.services.kinesis.AmazonKinesis;
25+
import com.amazonaws.services.kinesis.model.Shard;
26+
import org.junit.jupiter.api.Test;
27+
28+
import java.util.Arrays;
29+
import java.util.List;
30+
import java.util.Properties;
31+
32+
import static org.assertj.core.api.Assertions.assertThat;
33+
34+
/** Test for methods in the {@link DynamoDBStreamsProxy} class. */
35+
class DynamoDBStreamsProxyTest {
36+
37+
private static final String FAKE_STREAM_NAME = "fake-stream";
38+
39+
private static final List<String> SHARD_IDS =
40+
Arrays.asList(
41+
"shardId-000000000000",
42+
"shardId-000000000001",
43+
"shardId-000000000002",
44+
"shardId-000000000003");
45+
46+
@Test
47+
void testGetShardIterator() throws Exception {
48+
49+
String invalidShardId = "shardId-000000000004";
50+
51+
DynamoDBStreamsProxy ddbStreamsProxy = new TestableDynamoDBStreamsProxy();
52+
53+
for (String shardId : SHARD_IDS) {
54+
String shardIterator =
55+
ddbStreamsProxy.getShardIterator(getStreamShardHandle(shardId), "LATEST", null);
56+
57+
assertThat(shardIterator).isEqualTo("fakeShardIterator");
58+
}
59+
60+
String invalidShardIterator =
61+
ddbStreamsProxy.getShardIterator(
62+
getStreamShardHandle(invalidShardId), "LATEST", null);
63+
64+
assertThat(invalidShardIterator).isNull();
65+
}
66+
67+
private StreamShardHandle getStreamShardHandle(String shardId) {
68+
return new StreamShardHandle(FAKE_STREAM_NAME, new Shard().withShardId(shardId));
69+
}
70+
71+
private static class TestableDynamoDBStreamsProxy extends DynamoDBStreamsProxy {
72+
73+
private TestableDynamoDBStreamsProxy() {
74+
super(TestUtils.getStandardProperties());
75+
}
76+
77+
protected AmazonKinesis createKinesisClient(Properties configProps) {
78+
return FakeKinesisClientFactory.resourceNotFoundWhenGettingShardIterator(
79+
FAKE_STREAM_NAME, SHARD_IDS);
80+
}
81+
}
82+
}

0 commit comments

Comments
 (0)