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/05/03 14:28:17 UTC

[11/13] flink git commit: [FLINK-5969] Add KafkaConsumerBaseFrom12MigrationTest

[FLINK-5969] Add KafkaConsumerBaseFrom12MigrationTest

The binary snapshots were created on the Flink 1.2 branch.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/e40f2e18
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/e40f2e18
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/e40f2e18

Branch: refs/heads/master
Commit: e40f2e184c6e57d4346312f969a64727389e92fa
Parents: 0ecb5d0
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Apr 24 15:12:14 2017 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Wed May 3 16:25:57 2017 +0200

----------------------------------------------------------------------
 ...inkKafkaConsumerBaseFrom11MigrationTest.java | 347 +++++++++++++++++++
 ...inkKafkaConsumerBaseFrom12MigrationTest.java | 338 ++++++++++++++++++
 .../FlinkKafkaConsumerBaseMigrationTest.java    | 347 -------------------
 ...ka-consumer-migration-test-flink1.2-snapshot | Bin 0 -> 1022 bytes
 ...migration-test-flink1.2-snapshot-empty-state | Bin 0 -> 240 bytes
 5 files changed, 685 insertions(+), 347 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
new file mode 100644
index 0000000..7cc1f9c
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom11MigrationTest.java
@@ -0,0 +1,347 @@
+/*
+ * 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.kafka;
+
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
+import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+
+import static org.mockito.Mockito.mock;
+
+/**
+ * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
+ * done using the Flink 1.1 {@link FlinkKafkaConsumerBase}.
+ *
+ * <p>For regenerating the binary snapshot file you have to run the commented out portion
+ * of each test on a checkout of the Flink 1.1 branch.
+ */
+public class FlinkKafkaConsumerBaseFrom11MigrationTest {
+
+	/** Test restoring from an legacy empty state, when no partitions could be found for topics. */
+	@Test
+	public void testRestoreFromFlink11WithEmptyStateNoPartitions() throws Exception {
+		final DummyFlinkKafkaConsumer<String> consumerFunction =
+			new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file using legacy method
+		testHarness.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
+		testHarness.open();
+
+		// assert that no partitions were found and is empty
+		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// assert that no state was restored
+		Assert.assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/** Test restoring from an empty state taken using Flink 1.1, when some partitions could be found for topics. */
+	@Test
+	public void testRestoreFromFlink11WithEmptyStateWithPartitions() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
+		partitions.add(new KafkaTopicPartition("abc", 13));
+		partitions.add(new KafkaTopicPartition("def", 7));
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+			new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file using legacy method
+		testHarness.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
+		testHarness.open();
+
+		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot-empty-state";
+		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
+		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
+		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("abc", 13), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
+		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("def", 7), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
+
+		// assert that there are partitions and is identical to expected list
+		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		// assert that no state was restored
+		Assert.assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/** Test restoring from a non-empty state taken using Flink 1.1, when some partitions could be found for topics. */
+	@Test
+	public void testRestoreFromFlink11() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>();
+		partitions.add(new KafkaTopicPartition("abc", 13));
+		partitions.add(new KafkaTopicPartition("def", 7));
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+			new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file using legacy method
+		testHarness.initializeStateFromLegacyCheckpoint(
+			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot"));
+		testHarness.open();
+
+		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot"
+		final HashMap<KafkaTopicPartition, Long> expectedState = new HashMap<>();
+		expectedState.put(new KafkaTopicPartition("abc", 13), 16768L);
+		expectedState.put(new KafkaTopicPartition("def", 7), 987654321L);
+
+		// assert that there are partitions and is identical to expected list
+		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
+		Assert.assertEquals(expectedState, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		// assert that state is correctly restored from legacy checkpoint
+		Assert.assertTrue(consumerFunction.getRestoredState() != null);
+		Assert.assertEquals(expectedState, consumerFunction.getRestoredState());
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static String getResourceFilename(String filename) {
+		ClassLoader cl = FlinkKafkaConsumerBaseFrom11MigrationTest.class.getClassLoader();
+		URL resource = cl.getResource(filename);
+		if (resource == null) {
+			throw new NullPointerException("Missing snapshot resource.");
+		}
+		return resource.getFile();
+	}
+
+	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		private final List<KafkaTopicPartition> partitions;
+
+		@SuppressWarnings("unchecked")
+		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.partitions = partitions;
+		}
+
+		@Override
+		protected AbstractFetcher<T, ?> createFetcher(
+				SourceContext<T> sourceContext,
+				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
+				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+				StreamingRuntimeContext runtimeContext,
+				OffsetCommitMode offsetCommitMode) throws Exception {
+			return mock(AbstractFetcher.class);
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return partitions;
+		}
+
+		@Override
+		protected boolean getIsAutoCommitEnabled() {
+			return false;
+		}
+	}
+}
+
+/*
+	THE CODE FOR FLINK 1.1
+
+	@Test
+	public void testRestoreFromFlink11() throws Exception {
+		// --------------------------------------------------------------------
+		//   prepare fake states
+		// --------------------------------------------------------------------
+
+		final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
+		state1.put(new KafkaTopicPartition("abc", 13), 16768L);
+		state1.put(new KafkaTopicPartition("def", 7), 987654321L);
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		when(fetcher.snapshotCurrentState()).thenReturn(state1);
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
+			new FetcherFactory<String>() {
+				private static final long serialVersionUID = -2803131905656983619L;
+
+				@Override
+				public AbstractFetcher<String, ?> createFetcher() {
+					return fetcher;
+				}
+			});
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+			new StreamSource<>(consumerFunction);
+
+		final OneInputStreamOperatorTestHarness<Void, String> testHarness =
+			new OneInputStreamOperatorTestHarness<>(consumerOperator);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		testHarness.open();
+
+		final Throwable[] error = new Throwable[1];
+
+		// run the source asynchronously
+		Thread runner = new Thread() {
+			@Override
+			public void run() {
+				try {
+					consumerFunction.run(new DummySourceContext() {
+						@Override
+						public void collect(String element) {
+							latch.trigger();
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = t;
+				}
+			}
+		};
+		runner.start();
+
+		if (!latch.isTriggered()) {
+			latch.await();
+		}
+
+		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
+		testHarness.snaphotToFile(snapshot, "src/test/resources/kafka-consumer-migration-test-flink1.1-snapshot-2");
+		consumerOperator.run(new Object());
+
+		consumerOperator.close();
+		runner.join();
+
+		System.out.println("Killed");
+	}
+
+	private static abstract class DummySourceContext
+		implements SourceFunction.SourceContext<String> {
+
+		private final Object lock = new Object();
+
+		@Override
+		public void collectWithTimestamp(String element, long timestamp) {
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+		}
+	}
+
+
+	// ------------------------------------------------------------------------
+
+	private interface FetcherFactory<T> extends Serializable {
+		AbstractFetcher<T, ?> createFetcher();
+	}
+
+	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		private final FetcherFactory<T> fetcherFactory;
+
+		@SuppressWarnings("unchecked")
+		public DummyFlinkKafkaConsumer(FetcherFactory<T> fetcherFactory) {
+			super((KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+
+			final List<KafkaTopicPartition> partitions = new ArrayList<>();
+			partitions.add(new KafkaTopicPartition("dummy-topic", 0));
+			setSubscribedPartitions(partitions);
+
+			this.fetcherFactory = fetcherFactory;
+		}
+
+		@Override
+		protected AbstractFetcher<T, ?> createFetcher(SourceContext<T> sourceContext, List<KafkaTopicPartition> thisSubtaskPartitions, SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, StreamingRuntimeContext runtimeContext) throws Exception {
+			return fetcherFactory.createFetcher();
+		}
+	}
+* */

http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
new file mode 100644
index 0000000..6414a12
--- /dev/null
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java
@@ -0,0 +1,338 @@
+/*
+ * 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.kafka;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
+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.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
+ * done using the Flink 1.2 {@link FlinkKafkaConsumerBase}.
+ *
+ * <p>For regenerating the binary snapshot files run {@link #writeSnapshot()} on the Flink 1.2
+ * branch.
+ */
+public class FlinkKafkaConsumerBaseFrom12MigrationTest {
+
+	final static HashMap<KafkaTopicPartition, Long> PARTITION_STATE = new HashMap<>();
+
+	static {
+		PARTITION_STATE.put(new KafkaTopicPartition("abc", 13), 16768L);
+		PARTITION_STATE.put(new KafkaTopicPartition("def", 7), 987654321L);
+	}
+
+	/**
+	 * Manually run this to write binary snapshot data.
+	 */
+	@Ignore
+	@Test
+	public void writeSnapshot() throws Exception {
+		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", PARTITION_STATE);
+
+		final HashMap<KafkaTopicPartition, Long> emptyState = new HashMap<>();
+		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state", emptyState);
+	}
+
+	private void writeSnapshot(String path, HashMap<KafkaTopicPartition, Long> state) throws Exception {
+
+		final OneShotLatch latch = new OneShotLatch();
+		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
+
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) throws Throwable {
+				latch.trigger();
+				return null;
+			}
+		}).when(fetcher).runFetchLoop();
+
+		when(fetcher.snapshotCurrentState()).thenReturn(state);
+
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(fetcher, partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		testHarness.open();
+
+		final Throwable[] error = new Throwable[1];
+
+		// run the source asynchronously
+		Thread runner = new Thread() {
+			@Override
+			public void run() {
+				try {
+					consumerFunction.run(new DummySourceContext() {
+						@Override
+						public void collect(String element) {
+
+						}
+					});
+				}
+				catch (Throwable t) {
+					t.printStackTrace();
+					error[0] = 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();
+	}
+
+	/**
+	 * Test restoring from an legacy empty state, when no partitions could be found for topics.
+	 */
+	@Test
+	public void testRestoreFromEmptyStateNoPartitions() throws Exception {
+		final DummyFlinkKafkaConsumer<String> consumerFunction =
+				new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file
+		testHarness.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
+		testHarness.open();
+
+		// assert that no partitions were found and is empty
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// assert that no state was restored
+		assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/**
+	 * Test restoring from an empty state taken using Flink 1.2, when some partitions could be
+	 * found for topics.
+	 */
+	@Test
+	public void testRestoreFromEmptyStateWithPartitions() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file
+		testHarness.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
+		testHarness.open();
+
+		// the expected state in "kafka-consumer-migration-test-flink1.2-snapshot-empty-state";
+		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
+		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
+		for (KafkaTopicPartition partition : PARTITION_STATE.keySet()) {
+			expectedSubscribedPartitionsWithStartOffsets.put(partition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
+		}
+
+		// assert that there are partitions and is identical to expected list
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		assertTrue(consumerFunction.getRestoredState() == null);
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	/**
+	 * Test restoring from a non-empty state taken using Flink 1.2, when some partitions could be
+	 * found for topics.
+	 */
+	@Test
+	public void testRestore() throws Exception {
+		final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
+
+		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
+
+		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
+				new StreamSource<>(consumerFunction);
+
+		final AbstractStreamOperatorTestHarness<String> testHarness =
+				new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
+
+		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+
+		testHarness.setup();
+		// restore state from binary snapshot file
+		testHarness.initializeState(
+				OperatorSnapshotUtil.readStateHandle(
+						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
+		testHarness.open();
+
+		// assert that there are partitions and is identical to expected list
+		assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
+		assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
+
+		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
+		Assert.assertEquals(PARTITION_STATE, consumerFunction.getSubscribedPartitionsToStartOffsets());
+
+		// assert that state is correctly restored from legacy checkpoint
+		assertTrue(consumerFunction.getRestoredState() != null);
+		Assert.assertEquals(PARTITION_STATE, consumerFunction.getRestoredState());
+
+		consumerOperator.close();
+		consumerOperator.cancel();
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
+		private static final long serialVersionUID = 1L;
+
+		private final List<KafkaTopicPartition> partitions;
+
+		private final AbstractFetcher<T, ?> fetcher;
+
+		@SuppressWarnings("unchecked")
+		DummyFlinkKafkaConsumer(AbstractFetcher<T, ?> fetcher, List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcher = fetcher;
+			this.partitions = partitions;
+		}
+
+		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
+			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
+			this.fetcher = mock(AbstractFetcher.class);
+			this.partitions = partitions;
+		}
+
+		@Override
+		protected AbstractFetcher<T, ?> createFetcher(
+				SourceContext<T> sourceContext,
+				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
+				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
+				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
+				StreamingRuntimeContext runtimeContext,
+				OffsetCommitMode offsetCommitMode) throws Exception {
+			return fetcher;
+		}
+
+		@Override
+		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
+			return partitions;
+		}
+
+		@Override
+		protected boolean getIsAutoCommitEnabled() {
+			return false;
+		}
+	}
+
+
+	private static abstract class DummySourceContext
+			implements SourceFunction.SourceContext<String> {
+
+		private final Object lock = new Object();
+
+		@Override
+		public void collectWithTimestamp(String element, long timestamp) {
+		}
+
+		@Override
+		public void emitWatermark(Watermark mark) {
+		}
+
+		@Override
+		public Object getCheckpointLock() {
+			return lock;
+		}
+
+		@Override
+		public void close() {
+		}
+
+		@Override
+		public void markAsTemporarilyIdle() {
+
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
deleted file mode 100644
index 9fc261e..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * 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.kafka;
-
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode;
-import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
-import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionStateSentinel;
-import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
-import org.apache.flink.util.SerializedValue;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-
-import static org.mockito.Mockito.mock;
-
-/**
- * Tests for checking whether {@link FlinkKafkaConsumerBase} can restore from snapshots that were
- * done using the Flink 1.1 {@link FlinkKafkaConsumerBase}.
- *
- * <p>For regenerating the binary snapshot file you have to run the commented out portion
- * of each test on a checkout of the Flink 1.1 branch.
- */
-public class FlinkKafkaConsumerBaseMigrationTest {
-
-	/** Test restoring from an legacy empty state, when no partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateNoPartitions() throws Exception {
-		final DummyFlinkKafkaConsumer<String> consumerFunction =
-			new DummyFlinkKafkaConsumer<>(Collections.<KafkaTopicPartition>emptyList());
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator = new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
-		testHarness.open();
-
-		// assert that no partitions were found and is empty
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// assert that no state was restored
-		Assert.assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/** Test restoring from an empty state taken using Flink 1.1, when some partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11WithEmptyStateWithPartitions() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot-empty-state"));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot-empty-state";
-		// since the state is empty, the consumer should reflect on the startup mode to determine start offsets.
-		final HashMap<KafkaTopicPartition, Long> expectedSubscribedPartitionsWithStartOffsets = new HashMap<>();
-		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("abc", 13), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-		expectedSubscribedPartitionsWithStartOffsets.put(new KafkaTopicPartition("def", 7), KafkaTopicPartitionStateSentinel.GROUP_OFFSET);
-
-		// assert that there are partitions and is identical to expected list
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-		Assert.assertEquals(expectedSubscribedPartitionsWithStartOffsets, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		// assert that no state was restored
-		Assert.assertTrue(consumerFunction.getRestoredState() == null);
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	/** Test restoring from a non-empty state taken using Flink 1.1, when some partitions could be found for topics. */
-	@Test
-	public void testRestoreFromFlink11() throws Exception {
-		final List<KafkaTopicPartition> partitions = new ArrayList<>();
-		partitions.add(new KafkaTopicPartition("abc", 13));
-		partitions.add(new KafkaTopicPartition("def", 7));
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(partitions);
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final AbstractStreamOperatorTestHarness<String> testHarness =
-			new AbstractStreamOperatorTestHarness<>(consumerOperator, 1, 1, 0);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		// restore state from binary snapshot file using legacy method
-		testHarness.initializeStateFromLegacyCheckpoint(
-			getResourceFilename("kafka-consumer-migration-test-flink1.1-snapshot"));
-		testHarness.open();
-
-		// the expected state in "kafka-consumer-migration-test-flink1.1-snapshot"
-		final HashMap<KafkaTopicPartition, Long> expectedState = new HashMap<>();
-		expectedState.put(new KafkaTopicPartition("abc", 13), 16768L);
-		expectedState.put(new KafkaTopicPartition("def", 7), 987654321L);
-
-		// assert that there are partitions and is identical to expected list
-		Assert.assertTrue(consumerFunction.getSubscribedPartitionsToStartOffsets() != null);
-		Assert.assertTrue(!consumerFunction.getSubscribedPartitionsToStartOffsets().isEmpty());
-
-		// on restore, subscribedPartitionsToStartOffsets should be identical to the restored state
-		Assert.assertEquals(expectedState, consumerFunction.getSubscribedPartitionsToStartOffsets());
-
-		// assert that state is correctly restored from legacy checkpoint
-		Assert.assertTrue(consumerFunction.getRestoredState() != null);
-		Assert.assertEquals(expectedState, consumerFunction.getRestoredState());
-
-		consumerOperator.close();
-		consumerOperator.cancel();
-	}
-
-	// ------------------------------------------------------------------------
-
-	private static String getResourceFilename(String filename) {
-		ClassLoader cl = FlinkKafkaConsumerBaseMigrationTest.class.getClassLoader();
-		URL resource = cl.getResource(filename);
-		if (resource == null) {
-			throw new NullPointerException("Missing snapshot resource.");
-		}
-		return resource.getFile();
-	}
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final List<KafkaTopicPartition> partitions;
-
-		@SuppressWarnings("unchecked")
-		DummyFlinkKafkaConsumer(List<KafkaTopicPartition> partitions) {
-			super(Arrays.asList("dummy-topic"), (KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-			this.partitions = partitions;
-		}
-
-		@Override
-		protected AbstractFetcher<T, ?> createFetcher(
-				SourceContext<T> sourceContext,
-				Map<KafkaTopicPartition, Long> thisSubtaskPartitionsWithStartOffsets,
-				SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic,
-				SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated,
-				StreamingRuntimeContext runtimeContext,
-				OffsetCommitMode offsetCommitMode) throws Exception {
-			return mock(AbstractFetcher.class);
-		}
-
-		@Override
-		protected List<KafkaTopicPartition> getKafkaPartitions(List<String> topics) {
-			return partitions;
-		}
-
-		@Override
-		protected boolean getIsAutoCommitEnabled() {
-			return false;
-		}
-	}
-}
-
-/*
-	THE CODE FOR FLINK 1.1
-
-	@Test
-	public void testRestoreFromFlink11() throws Exception {
-		// --------------------------------------------------------------------
-		//   prepare fake states
-		// --------------------------------------------------------------------
-
-		final HashMap<KafkaTopicPartition, Long> state1 = new HashMap<>();
-		state1.put(new KafkaTopicPartition("abc", 13), 16768L);
-		state1.put(new KafkaTopicPartition("def", 7), 987654321L);
-
-		final OneShotLatch latch = new OneShotLatch();
-		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
-
-		doAnswer(new Answer<Void>() {
-			@Override
-			public Void answer(InvocationOnMock invocation) throws Throwable {
-				latch.trigger();
-				return null;
-			}
-		}).when(fetcher).runFetchLoop();
-
-		when(fetcher.snapshotCurrentState()).thenReturn(state1);
-
-		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
-			new FetcherFactory<String>() {
-				private static final long serialVersionUID = -2803131905656983619L;
-
-				@Override
-				public AbstractFetcher<String, ?> createFetcher() {
-					return fetcher;
-				}
-			});
-
-		StreamSource<String, DummyFlinkKafkaConsumer<String>> consumerOperator =
-			new StreamSource<>(consumerFunction);
-
-		final OneInputStreamOperatorTestHarness<Void, String> testHarness =
-			new OneInputStreamOperatorTestHarness<>(consumerOperator);
-
-		testHarness.setTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-
-		testHarness.setup();
-		testHarness.open();
-
-		final Throwable[] error = new Throwable[1];
-
-		// run the source asynchronously
-		Thread runner = new Thread() {
-			@Override
-			public void run() {
-				try {
-					consumerFunction.run(new DummySourceContext() {
-						@Override
-						public void collect(String element) {
-							latch.trigger();
-						}
-					});
-				}
-				catch (Throwable t) {
-					t.printStackTrace();
-					error[0] = t;
-				}
-			}
-		};
-		runner.start();
-
-		if (!latch.isTriggered()) {
-			latch.await();
-		}
-
-		StreamTaskState snapshot = testHarness.snapshot(0L, 0L);
-		testHarness.snaphotToFile(snapshot, "src/test/resources/kafka-consumer-migration-test-flink1.1-snapshot-2");
-		consumerOperator.run(new Object());
-
-		consumerOperator.close();
-		runner.join();
-
-		System.out.println("Killed");
-	}
-
-	private static abstract class DummySourceContext
-		implements SourceFunction.SourceContext<String> {
-
-		private final Object lock = new Object();
-
-		@Override
-		public void collectWithTimestamp(String element, long timestamp) {
-		}
-
-		@Override
-		public void emitWatermark(Watermark mark) {
-		}
-
-		@Override
-		public Object getCheckpointLock() {
-			return lock;
-		}
-
-		@Override
-		public void close() {
-		}
-	}
-
-
-	// ------------------------------------------------------------------------
-
-	private interface FetcherFactory<T> extends Serializable {
-		AbstractFetcher<T, ?> createFetcher();
-	}
-
-	private static class DummyFlinkKafkaConsumer<T> extends FlinkKafkaConsumerBase<T> {
-		private static final long serialVersionUID = 1L;
-
-		private final FetcherFactory<T> fetcherFactory;
-
-		@SuppressWarnings("unchecked")
-		public DummyFlinkKafkaConsumer(FetcherFactory<T> fetcherFactory) {
-			super((KeyedDeserializationSchema< T >) mock(KeyedDeserializationSchema.class));
-
-			final List<KafkaTopicPartition> partitions = new ArrayList<>();
-			partitions.add(new KafkaTopicPartition("dummy-topic", 0));
-			setSubscribedPartitions(partitions);
-
-			this.fetcherFactory = fetcherFactory;
-		}
-
-		@Override
-		protected AbstractFetcher<T, ?> createFetcher(SourceContext<T> sourceContext, List<KafkaTopicPartition> thisSubtaskPartitions, SerializedValue<AssignerWithPeriodicWatermarks<T>> watermarksPeriodic, SerializedValue<AssignerWithPunctuatedWatermarks<T>> watermarksPunctuated, StreamingRuntimeContext runtimeContext) throws Exception {
-			return fetcherFactory.createFetcher();
-		}
-	}
-* */

http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot
new file mode 100644
index 0000000..f0be11a
Binary files /dev/null and b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot differ

http://git-wip-us.apache.org/repos/asf/flink/blob/e40f2e18/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state
new file mode 100644
index 0000000..45047ee
Binary files /dev/null and b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot-empty-state differ