-
Notifications
You must be signed in to change notification settings - Fork 13.9k
[FLINK-17170][kinesis] Fix deadlock during stop-with-savepoint. #15826
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
Merged
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
157 changes: 157 additions & 0 deletions
157
...nesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisITCase.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,157 @@ | ||
| /* | ||
| * 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.streaming.connectors.kinesis; | ||
|
|
||
| import org.apache.flink.api.common.functions.MapFunction; | ||
| import org.apache.flink.api.common.serialization.SimpleStringSchema; | ||
| import org.apache.flink.runtime.client.JobStatusMessage; | ||
| import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition; | ||
| import org.apache.flink.streaming.connectors.kinesis.testutils.KinesaliteContainer; | ||
| import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisPubsubClient; | ||
| import org.apache.flink.test.util.MiniClusterWithClientResource; | ||
|
|
||
| import org.junit.Before; | ||
| import org.junit.ClassRule; | ||
| import org.junit.Rule; | ||
| import org.junit.Test; | ||
| import org.junit.rules.TemporaryFolder; | ||
| import org.testcontainers.utility.DockerImageName; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Properties; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.ForkJoinPool; | ||
| import java.util.concurrent.ForkJoinTask; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
|
|
||
| import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_INITIAL_POSITION; | ||
| import static org.hamcrest.Matchers.equalTo; | ||
| import static org.hamcrest.Matchers.hasSize; | ||
| import static org.hamcrest.Matchers.lessThan; | ||
| import static org.junit.Assert.assertThat; | ||
|
|
||
| /** IT cases for using Kinesis consumer/producer based on Kinesalite. */ | ||
| public class FlinkKinesisITCase { | ||
| public static final String TEST_STREAM = "test_stream"; | ||
|
|
||
| @ClassRule | ||
| public static MiniClusterWithClientResource miniCluster = | ||
| new MiniClusterWithClientResource( | ||
| new MiniClusterResourceConfiguration.Builder().build()); | ||
|
|
||
| @ClassRule | ||
| public static KinesaliteContainer kinesalite = | ||
| new KinesaliteContainer( | ||
| DockerImageName.parse("instructure/kinesalite").withTag("latest")); | ||
|
|
||
| @Rule public TemporaryFolder temp = new TemporaryFolder(); | ||
|
|
||
| private static final SimpleStringSchema STRING_SCHEMA = new SimpleStringSchema(); | ||
|
|
||
| private KinesisPubsubClient client; | ||
|
|
||
| @Before | ||
| public void setupClient() { | ||
| client = new KinesisPubsubClient(kinesalite.getContainerProperties()); | ||
| } | ||
|
|
||
| /** | ||
| * Tests that pending elements do not cause a deadlock during stop with savepoint (FLINK-17170). | ||
| * | ||
| * <ol> | ||
| * <li>The test setups up a stream with 100 records and creates a Flink job that reads them | ||
| * with very slowly (using up a large chunk of time of the mailbox). | ||
| * <li>After ensuring that consumption has started, the job is stopped in a parallel thread. | ||
| * <li>Without the fix of FLINK-17170, the job now has a high chance to deadlock during | ||
| * cancel. | ||
| * <li>With the fix, the job proceeds and we can lift the backpressure. | ||
| * </ol> | ||
| */ | ||
| @Test | ||
| public void testStopWithSavepoint() throws Exception { | ||
| client.createTopic(TEST_STREAM, 1, new Properties()); | ||
|
|
||
| // add elements to the test stream | ||
| int numElements = 10; | ||
| List<String> elements = | ||
| IntStream.range(0, numElements) | ||
| .mapToObj(String::valueOf) | ||
| .collect(Collectors.toList()); | ||
| for (String element : elements) { | ||
| client.sendMessage(TEST_STREAM, element); | ||
| } | ||
|
|
||
| StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); | ||
| env.setParallelism(1); | ||
|
|
||
| Properties config = kinesalite.getContainerProperties(); | ||
| config.setProperty(STREAM_INITIAL_POSITION, InitialPosition.TRIM_HORIZON.name()); | ||
| FlinkKinesisConsumer<String> consumer = | ||
| new FlinkKinesisConsumer<>(TEST_STREAM, STRING_SCHEMA, config); | ||
|
|
||
| // call stop with savepoint in another thread | ||
| ForkJoinTask<Object> stopTask = | ||
| ForkJoinPool.commonPool() | ||
| .submit( | ||
| () -> { | ||
| WaitingMapper.firstElement.await(); | ||
| stopWithSavepoint(); | ||
| WaitingMapper.stopped = true; | ||
| return null; | ||
| }); | ||
|
|
||
| try { | ||
| List<String> result = | ||
| env.addSource(consumer).map(new WaitingMapper()).executeAndCollect(10000); | ||
| // stop with savepoint will most likely only return a small subset of the elements | ||
| // validate that the prefix is as expected | ||
| assertThat(result, hasSize(lessThan(numElements))); | ||
| assertThat(result, equalTo(elements.subList(0, result.size()))); | ||
| } finally { | ||
| stopTask.cancel(true); | ||
| } | ||
| } | ||
|
|
||
| private String stopWithSavepoint() throws Exception { | ||
| JobStatusMessage job = | ||
| miniCluster.getClusterClient().listJobs().get().stream().findFirst().get(); | ||
| return miniCluster | ||
| .getClusterClient() | ||
| .stopWithSavepoint(job.getJobId(), true, temp.getRoot().getAbsolutePath()) | ||
| .get(); | ||
| } | ||
|
|
||
| private static class WaitingMapper implements MapFunction<String, String> { | ||
| static CountDownLatch firstElement = new CountDownLatch(1); | ||
| static volatile boolean stopped = false; | ||
|
|
||
| @Override | ||
| public String map(String value) throws Exception { | ||
| if (firstElement.getCount() > 0) { | ||
| firstElement.countDown(); | ||
| } | ||
| if (!stopped) { | ||
| Thread.sleep(100); | ||
| } | ||
| return value; | ||
| } | ||
| } | ||
| } | ||
137 changes: 137 additions & 0 deletions
137
...est/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesaliteContainer.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,137 @@ | ||
| /* | ||
| * 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.streaming.connectors.kinesis.testutils; | ||
|
|
||
| import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; | ||
|
|
||
| import com.amazonaws.auth.AWSStaticCredentialsProvider; | ||
| import com.amazonaws.auth.BasicAWSCredentials; | ||
| import com.amazonaws.client.builder.AwsClientBuilder; | ||
| import com.amazonaws.services.kinesis.AmazonKinesis; | ||
| import com.amazonaws.services.kinesis.AmazonKinesisClientBuilder; | ||
| import com.amazonaws.services.kinesis.model.ListStreamsResult; | ||
| import org.rnorth.ducttape.unreliables.Unreliables; | ||
| import org.testcontainers.containers.GenericContainer; | ||
| import org.testcontainers.containers.wait.strategy.AbstractWaitStrategy; | ||
| import org.testcontainers.utility.DockerImageName; | ||
|
|
||
| import java.util.Properties; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| import static com.amazonaws.SDKGlobalConfiguration.ACCESS_KEY_ENV_VAR; | ||
| import static com.amazonaws.SDKGlobalConfiguration.SECRET_KEY_ENV_VAR; | ||
|
|
||
| /** | ||
| * A testcontainer based on Kinesalite. | ||
| * | ||
| * <p>Note that the more obvious localstack container with Kinesis took 1 minute to start vs 10 | ||
| * seconds of Kinesalite. | ||
| */ | ||
| public class KinesaliteContainer extends GenericContainer<KinesaliteContainer> { | ||
| private static final String ACCESS_KEY = "access key"; | ||
| private static final String SECRET_KEY = "secret key"; | ||
|
|
||
| public KinesaliteContainer(DockerImageName imageName) { | ||
| super(imageName); | ||
|
|
||
| withEnv(ACCESS_KEY_ENV_VAR, ACCESS_KEY); | ||
| withEnv(SECRET_KEY_ENV_VAR, ACCESS_KEY); | ||
| withExposedPorts(4567); | ||
| waitingFor(new ListStreamsWaitStrategy()); | ||
| withCreateContainerCmdModifier( | ||
| cmd -> | ||
| cmd.withEntrypoint( | ||
| "/tini", | ||
| "--", | ||
| "/usr/src/app/node_modules/kinesalite/cli.js", | ||
| "--path", | ||
| "/var/lib/kinesalite", | ||
| "--ssl")); | ||
| } | ||
|
|
||
| /** Returns the endpoint url to access the container from outside the docker network. */ | ||
| public String getContainerEndpointUrl() { | ||
| return String.format("https://%s:%s", getContainerIpAddress(), getMappedPort(4567)); | ||
| } | ||
|
|
||
| /** Returns the endpoint url to access the host from inside the docker network. */ | ||
| public String getHostEndpointUrl() { | ||
| return String.format("https://%s:%s", getHost(), getMappedPort(4567)); | ||
| } | ||
|
|
||
| public String getAccessKey() { | ||
| return ACCESS_KEY; | ||
| } | ||
|
|
||
| public String getSecretKey() { | ||
| return SECRET_KEY; | ||
| } | ||
|
|
||
| /** Returns the properties to access the container from outside the docker network. */ | ||
| public Properties getContainerProperties() { | ||
| return getProperties(getContainerEndpointUrl()); | ||
| } | ||
|
|
||
| /** Returns the properties to access the host from inside the docker network. */ | ||
| public Properties getHostProperties() { | ||
| return getProperties(getHostEndpointUrl()); | ||
| } | ||
|
|
||
| /** Returns the client to access the container from outside the docker network. */ | ||
| public AmazonKinesis getContainerClient() { | ||
| return getClient(getContainerEndpointUrl()); | ||
| } | ||
|
|
||
| /** Returns the client to access the host from inside the docker network. */ | ||
| public AmazonKinesis getHostClient() { | ||
| return getClient(getHostEndpointUrl()); | ||
| } | ||
|
|
||
| private AmazonKinesis getClient(String endPoint) { | ||
| return AmazonKinesisClientBuilder.standard() | ||
| .withCredentials( | ||
| new AWSStaticCredentialsProvider( | ||
| new BasicAWSCredentials(getAccessKey(), getSecretKey()))) | ||
| .withEndpointConfiguration( | ||
| new AwsClientBuilder.EndpointConfiguration(endPoint, "us-east-1")) | ||
| .build(); | ||
| } | ||
|
|
||
| private Properties getProperties(String endpointUrl) { | ||
| Properties config = new Properties(); | ||
| config.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); | ||
| config.setProperty(AWSConfigConstants.AWS_ENDPOINT, endpointUrl); | ||
| config.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, getAccessKey()); | ||
| config.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, getSecretKey()); | ||
| return config; | ||
| } | ||
|
|
||
| private class ListStreamsWaitStrategy extends AbstractWaitStrategy { | ||
| @Override | ||
| protected void waitUntilReady() { | ||
| Unreliables.retryUntilSuccess( | ||
| (int) this.startupTimeout.getSeconds(), | ||
| TimeUnit.SECONDS, | ||
| () -> this.getRateLimiter().getWhenReady(() -> tryList())); | ||
| } | ||
|
|
||
| private ListStreamsResult tryList() { | ||
| return getContainerClient().listStreams(); | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.