Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-29324] [Connectors/Kinesis] Fix NPE for Kinesis connector when closing #20853

Merged
merged 2 commits into from Sep 20, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -418,7 +418,12 @@ public void cancel() {
public void close() throws Exception {
cancel();
// safe-guard when the fetcher has been interrupted, make sure to not leak resources
fetcher.awaitTermination();
// application might be stopped before connector subtask has been started
// so we must check if the fetcher is actually created
KinesisDataFetcher fetcher = this.fetcher;
if (fetcher != null) {
fetcher.awaitTermination();
}
this.fetcher = null;
super.close();
}
Expand Down
Expand Up @@ -1184,6 +1184,14 @@ public void emitWatermark(Watermark mark) {
testHarness.close();
}

@Test
public void testCloseConnectorBeforeSubtaskStart() throws Exception {
Properties config = TestUtils.getStandardProperties();
FlinkKinesisConsumer<String> consumer =
new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config);
consumer.close();
}

private void awaitRecordCount(ConcurrentLinkedQueue<? extends Object> queue, int count)
throws Exception {
Deadline deadline = Deadline.fromNow(Duration.ofSeconds(10));
Expand Down