You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by al...@apache.org on 2017/08/25 12:33:07 UTC
[1/2] flink git commit: [FLINK-7429] [kinesis] Add IT tests for
migration from 1.3
Repository: flink
Updated Branches:
refs/heads/master 6642768ad -> c384e52e6
[FLINK-7429] [kinesis] Add IT tests for migration from 1.3
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c384e52e
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c384e52e
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c384e52e
Branch: refs/heads/master
Commit: c384e52e647da457ce5127863148d03c93c1a4aa
Parents: 04add8d
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Fri Aug 18 11:27:38 2017 +0800
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Fri Aug 25 11:13:34 2017 +0200
----------------------------------------------------------------------
.../FlinkKinesisConsumerMigrationTest.java | 238 +++++++++++++++++++
...sumer-migration-test-flink1.3-empty-snapshot | Bin 0 -> 13975 bytes
...is-consumer-migration-test-flink1.3-snapshot | Bin 0 -> 14043 bytes
3 files changed, 238 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c384e52e/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
new file mode 100644
index 0000000..364560c
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerMigrationTest.java
@@ -0,0 +1,238 @@
+/*
+ * 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.RuntimeContext;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardMetadata;
+import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
+import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.OperatorSnapshotUtil;
+import org.apache.flink.streaming.util.migration.MigrationTestUtil;
+import org.apache.flink.streaming.util.migration.MigrationVersion;
+
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for checking whether {@link FlinkKinesisConsumer} can restore from snapshots that were
+ * done using an older {@code FlinkKinesisConsumer}.
+ *
+ * <p>For regenerating the binary snapshot files run {@link #writeSnapshot()} on the corresponding
+ * Flink release-* branch.
+ */
+@RunWith(Parameterized.class)
+public class FlinkKinesisConsumerMigrationTest {
+
+ /**
+ * TODO change this to the corresponding savepoint version to be written (e.g. {@link MigrationVersion#v1_3} for 1.3)
+ * TODO and remove all @Ignore annotations on the writeSnapshot() method to generate savepoints
+ */
+ private final MigrationVersion flinkGenerateSavepointVersion = null;
+
+ private static final HashMap<StreamShardMetadata, SequenceNumber> TEST_STATE = new HashMap<>();
+ static {
+ StreamShardMetadata shardMetadata = new StreamShardMetadata();
+ shardMetadata.setStreamName("fakeStream1");
+ shardMetadata.setShardId(KinesisShardIdGenerator.generateFromShardOrder(0));
+
+ TEST_STATE.put(shardMetadata, new SequenceNumber("987654321"));
+ }
+
+ private final MigrationVersion testMigrateVersion;
+
+ @Parameterized.Parameters(name = "Migration Savepoint: {0}")
+ public static Collection<MigrationVersion> parameters () {
+ return Arrays.asList(MigrationVersion.v1_3);
+ }
+
+ public FlinkKinesisConsumerMigrationTest(MigrationVersion testMigrateVersion) {
+ this.testMigrateVersion = testMigrateVersion;
+ }
+
+ /**
+ * Manually run this to write binary snapshot data.
+ */
+ @Ignore
+ @Test
+ public void writeSnapshot() throws Exception {
+ writeSnapshot("src/test/resources/kinesis-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-snapshot", TEST_STATE);
+
+ // write empty state snapshot
+ writeSnapshot("src/test/resources/kinesis-consumer-migration-test-flink" + flinkGenerateSavepointVersion + "-empty-snapshot", new HashMap<>());
+ }
+
+ @Test
+ public void testRestoreWithEmptyState() throws Exception {
+ final DummyFlinkKinesisConsumer<String> consumerFunction = new DummyFlinkKinesisConsumer<>(mock(KinesisDataFetcher.class));
+
+ StreamSource<String, DummyFlinkKinesisConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
+
+ final AbstractStreamOperatorTestHarness<String> testHarness =
+ new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+ testHarness.setup();
+ MigrationTestUtil.restoreFromSnapshot(
+ testHarness,
+ "src/test/resources/kinesis-consumer-migration-test-flink" + testMigrateVersion + "-empty-snapshot", testMigrateVersion);
+ testHarness.open();
+
+ // assert that no state was restored
+ assertTrue(consumerFunction.getRestoredState().isEmpty());
+
+ consumerOperator.close();
+ consumerOperator.cancel();
+ }
+
+ @Test
+ public void testRestore() throws Exception {
+ final DummyFlinkKinesisConsumer<String> consumerFunction = new DummyFlinkKinesisConsumer<>(mock(KinesisDataFetcher.class));
+
+ StreamSource<String, DummyFlinkKinesisConsumer<String>> consumerOperator =
+ new StreamSource<>(consumerFunction);
+
+ final AbstractStreamOperatorTestHarness<String> testHarness =
+ new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+ testHarness.setup();
+ MigrationTestUtil.restoreFromSnapshot(
+ testHarness,
+ "src/test/resources/kinesis-consumer-migration-test-flink" + testMigrateVersion + "-snapshot", testMigrateVersion);
+ testHarness.open();
+
+ // assert that state is correctly restored
+ assertNotEquals(null, consumerFunction.getRestoredState());
+ assertEquals(1, consumerFunction.getRestoredState().size());
+ assertEquals(TEST_STATE, consumerFunction.getRestoredState());
+
+ consumerOperator.close();
+ consumerOperator.cancel();
+ }
+
+ // ------------------------------------------------------------------------
+
+ @SuppressWarnings("unchecked")
+ private void writeSnapshot(String path, HashMap<StreamShardMetadata, SequenceNumber> state) throws Exception {
+ final OneShotLatch latch = new OneShotLatch();
+
+ final KinesisDataFetcher<String> fetcher = mock(KinesisDataFetcher.class);
+ doAnswer(new Answer() {
+ @Override
+ public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+ latch.trigger();
+ return null;
+ }
+ }).when(fetcher).runFetcher();
+ when(fetcher.snapshotState()).thenReturn(state);
+
+ final DummyFlinkKinesisConsumer<String> consumer = new DummyFlinkKinesisConsumer<>(fetcher);
+
+ StreamSource<String, DummyFlinkKinesisConsumer<String>> consumerOperator = new StreamSource<>(consumer);
+
+ final AbstractStreamOperatorTestHarness<String> testHarness =
+ new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+ testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+ testHarness.setup();
+ testHarness.open();
+
+ final AtomicReference<Throwable> error = new AtomicReference<>();
+
+ // run the source asynchronously
+ Thread runner = new Thread() {
+ @Override
+ public void run() {
+ try {
+ consumer.run(mock(SourceFunction.SourceContext.class));
+ } catch (Throwable t) {
+ t.printStackTrace();
+ error.set(t);
+ }
+ }
+ };
+ runner.start();
+
+ if (!latch.isTriggered()) {
+ latch.await();
+ }
+
+ final OperatorStateHandles snapshot;
+ synchronized (testHarness.getCheckpointLock()) {
+ snapshot = testHarness.snapshot(0L, 0L);
+ }
+
+ OperatorSnapshotUtil.writeStateHandle(snapshot, path);
+
+ consumerOperator.close();
+ runner.join();
+ }
+
+ private static class DummyFlinkKinesisConsumer<T> extends FlinkKinesisConsumer<T> {
+
+ private KinesisDataFetcher<T> mockFetcher;
+
+ private static Properties dummyConfig = new Properties();
+ static {
+ dummyConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1");
+ dummyConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId");
+ dummyConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey");
+ }
+
+ DummyFlinkKinesisConsumer(KinesisDataFetcher<T> mockFetcher) {
+ super("dummy-topic", mock(KinesisDeserializationSchema.class), dummyConfig);
+ this.mockFetcher = mockFetcher;
+ }
+
+ @Override
+ protected KinesisDataFetcher<T> createFetcher(
+ List<String> streams,
+ SourceContext<T> sourceContext,
+ RuntimeContext runtimeContext,
+ Properties configProps,
+ KinesisDeserializationSchema<T> deserializer) {
+ return mockFetcher;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/c384e52e/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot
new file mode 100644
index 0000000..aa981c0
Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-empty-snapshot differ
http://git-wip-us.apache.org/repos/asf/flink/blob/c384e52e/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot
new file mode 100644
index 0000000..ddf8a4d
Binary files /dev/null and b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.3-snapshot differ
[2/2] flink git commit: [hotfix] Remove old Kinesis snapshots for
Flink 1.1
Posted by al...@apache.org.
[hotfix] Remove old Kinesis snapshots for Flink 1.1
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/04add8d9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/04add8d9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/04add8d9
Branch: refs/heads/master
Commit: 04add8d9121807838f7ee82576a594ecbdfbd538
Parents: 6642768
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Fri Aug 25 11:12:40 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Fri Aug 25 11:13:34 2017 +0200
----------------------------------------------------------------------
...-consumer-migration-test-flink1.1-empty-snapshot | Bin 468 -> 0 bytes
...inesis-consumer-migration-test-flink1.1-snapshot | Bin 1140 -> 0 bytes
2 files changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/04add8d9/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot
deleted file mode 100644
index f4dd96d..0000000
Binary files a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-empty-snapshot and /dev/null differ
http://git-wip-us.apache.org/repos/asf/flink/blob/04add8d9/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot b/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot
deleted file mode 100644
index b60402e..0000000
Binary files a/flink-connectors/flink-connector-kinesis/src/test/resources/kinesis-consumer-migration-test-flink1.1-snapshot and /dev/null differ