You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by aljoscha <gi...@git.apache.org> on 2017/04/26 09:55:21 UTC

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

GitHub user aljoscha opened a pull request:

    https://github.com/apache/flink/pull/3778

    [FLINK-5969] Add savepoint backwards compatibility tests from 1.2 to 1.3

    The binary savepoints and snapshots in the tests were created on the commit of the Flink 1.2.0 release, so we test backwards compatibility within the Flink 1.2.x line. Once this is approved I'll open another PR that transplants these commits on the master branch (with the binary snapshots/savepoints done on Flink 1.2.0) so that we test migration compatibility between 1.2.0 and what is going to be Flink 1.3.x.
    
    I changed the naming of some existing tests so we now have `*From11MigrationTest` and `*From12MigrationTest` (and one ITCase). Immediately after releasing Flink 1.3.0 we should do the same, i.e. introduce `*From13MigrationTest` and ITCase based on the existing tests.
    
    The unit tests are somewhat straightforward: we feed some data into an operator using an operator test harness, then we do a snapshot. (This is the part that has to be done on the "old" version to generate the binary snapshot that goes into the repo). The actual tests restore an operator form that snapshot and verify the output.
    
    The ITCase is a bit more involved. We have a complete Job of user-functions and custom operators that tries to cover as many state/timer combinations as possible. We start the job and, using accumulators, observe the number of received elements in the sink. Once we get all elements we perform a savepoint and cancel the job. Thus we have all state caused by the elements reflected in our savepoint. This has to be done on the "old" version and the savepoint goes into the repo. The restoring job is instrumented with code that verifies restored state and updates accumulators. We listen on the accumulator changes and cancel the job once we have seen all required verifications.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/aljoscha/flink jira-5969-backwards-compat-12-13-on-release12

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3778.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3778
    
----
commit ef9e73a1f8af8903b0689eada2a9d853034fab88
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-20T12:48:22Z

    [FLINK-5969] Augment SavepointMigrationTestBase to catch failed jobs

commit 47143ba424355b7d25e9990bc308ea1744a0f33e
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-20T15:09:00Z

    [FLINK-5969] Add savepoint IT case that checks restore from 1.2
    
    The binary savepoints in this were created on the Flink 1.2.0 release
    commit.

commit 3803dc04caae5e57f2cb23df0b6bc4663f8af08e
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-21T09:43:53Z

    [FLINK-6353] Fix legacy user-state restore from 1.2
    
    State that was checkpointed using Checkpointed (on a user function)
    could be restored using CheckpointedRestoring when the savepoint was
    done on Flink 1.2. The reason was an overzealous check in
    AbstractUdfStreamOperator that only restores from "legacy" operator
    state using CheckpointedRestoring when the stream is a Migration stream.
    
    This removes that check but we still need to make sure to read away the
    byte that indicates whether there is legacy state, which is written when
    we're restoring from a Flink 1.1 savepoint.
    
    After this fix, the procedure for a user to migrate a user function away
    from the Checkpointed interface is this:
    
     - Perform savepoint with user function still implementing Checkpointed,
       shutdown job
     - Change user function to implement CheckpointedRestoring
     - Restore from previous savepoint, user function has to somehow move
       the state that is restored using CheckpointedRestoring to another
       type of state, .e.g operator state, using the OperatorStateStore.
     - Perform another savepoint, shutdown job
     - Remove CheckpointedRestoring interface from user function
     - Restore from the second savepoint
     - Done.
    
    If the CheckpointedRestoring interface is not removed as prescribed in
    the last steps then a future restore of a new savepoint will fail
    because Flink will try to read legacy operator state that is not there
    anymore.  The above steps also apply to Flink 1.3, when a user want's to
    move away from the Checkpointed interface.

commit f08661adcf3a64daf955ace70683ef2fe14cec2c
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T09:25:32Z

    [FLINK-5969] Add ContinuousFileProcessingFrom12MigrationTest
    
    The binary snapshots were created on the Flink 1.2 branch.

commit e70424eb6c9861e89c78f12143f319ce6eea49c1
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T10:31:53Z

    [FLINK-5969] Add OperatorSnapshotUtil
    
    This has methods for storing/reading OperatorStateHandles, as returned
    from stream operator test harnesses. This can be used to write binary
    snapshots for use in state migration tests.

commit 0217a2c3273157d4da936056fa5c76237d67b355
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T13:12:14Z

    [FLINK-5969] Add KafkaConsumerBaseFrom12MigrationTest
    
    The binary snapshots were created on the Flink 1.2 branch.

commit 6d3386bdb57e74ffecab76db211692aa734edf52
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-25T10:05:22Z

    [FLINK-5969] Rename StatefulUDFSavepointFrom*MigrationITCases

commit f63e52c367bf85d328b9b6b3913ffe7dbd935d11
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T15:13:27Z

    [FLINK-5969] Add WindowOperatorFrom12MigrationTest
    
    The binary snapshots for this were created on the Flink 1.2 branch.

commit 525f98de5a90752918c7620ffaf2490d9c540452
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T15:13:49Z

    [FLINK-5969] Also snapshot legacy state in operator test harness

commit 84fd38670dacf9f445f4361e85a494ad7512c3df
Author: Aljoscha Krettek <al...@gmail.com>
Date:   2017-04-24T15:50:59Z

    [FLINK-5969] Add BucketingSinkFrom12MigrationTest
    
    The binary snapshots have been created on the Flink 1.2 branch.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114353483
  
    --- Diff: flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java ---
    @@ -0,0 +1,224 @@
    +/*
    + * 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.fs.bucketing;
    +
    +import static org.junit.Assert.assertTrue;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.OperatorStateStore;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.connectors.fs.StringWriter;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.hadoop.fs.Path;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +/**
    + * Tests for checking whether {@link BucketingSink} can restore from snapshots that were done
    + * using the Flink 1.2 {@link BucketingSink}.
    + *
    + * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
    + * the Flink 1.2 branch.
    + */
    +
    +public class BucketingSinkFrom12MigrationTest {
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	private static final String PART_PREFIX = "part";
    +	private static final String PENDING_SUFFIX = ".pending";
    +	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
    +	private static final String VALID_LENGTH_SUFFIX = ".valid";
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +
    +		final File outDir = tempFolder.newFolder();
    +
    +		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
    +			.setWriter(new StringWriter<String>())
    +			.setBatchSize(5)
    +			.setPartPrefix(PART_PREFIX)
    +			.setInProgressPrefix("")
    +			.setPendingPrefix("")
    +			.setValidLengthPrefix("")
    +			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
    +			.setPendingSuffix(PENDING_SUFFIX)
    +			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
    +
    +		OneInputStreamOperatorTestHarness<String, Object> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
    +
    +		testHarness.setup();
    +		testHarness.open();
    +
    +		testHarness.processElement(new StreamRecord<>("test1", 0L));
    +		testHarness.processElement(new StreamRecord<>("test2", 0L));
    +
    +		checkFs(outDir, 1, 1, 0, 0);
    +
    +		testHarness.processElement(new StreamRecord<>("test3", 0L));
    +		testHarness.processElement(new StreamRecord<>("test4", 0L));
    +		testHarness.processElement(new StreamRecord<>("test5", 0L));
    +
    +		checkFs(outDir, 1, 4, 0, 0);
    +
    +		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot");
    +		testHarness.close();
    +	}
    +
    +	@Test
    +	public void testRestore() throws Exception {
    +		final File outDir = tempFolder.newFolder();
    +
    +		ValidatingBucketingSink<String> sink = (ValidatingBucketingSink<String>) new ValidatingBucketingSink<String>(outDir.getAbsolutePath())
    +			.setWriter(new StringWriter<String>())
    +			.setBatchSize(5)
    +			.setPartPrefix(PART_PREFIX)
    +			.setInProgressPrefix("")
    +			.setPendingPrefix("")
    +			.setValidLengthPrefix("")
    +			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
    +			.setPendingSuffix(PENDING_SUFFIX)
    +			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
    +
    +		OneInputStreamOperatorTestHarness<String, Object> testHarness = new OneInputStreamOperatorTestHarness<>(
    +			new StreamSink<>(sink), 10, 1, 0);
    +		testHarness.setup();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("bucketing-sink-migration-test-flink1.2-snapshot")));
    +		testHarness.open();
    +
    +		assertTrue(sink.initializeCalled);
    +
    +		testHarness.processElement(new StreamRecord<>("test1", 0L));
    +		testHarness.processElement(new StreamRecord<>("test2", 0L));
    +
    +		checkFs(outDir, 1, 1, 0, 0);
    +
    +		testHarness.close();
    +	}
    +
    +	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
    +		int inProg = 0;
    +		int pend = 0;
    +		int compl = 0;
    +		int val = 0;
    +
    +		for (File file: FileUtils.listFiles(outDir, null, true)) {
    +			if (file.getAbsolutePath().endsWith("crc")) {
    +				continue;
    +			}
    +			String path = file.getPath();
    +			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
    +				inProg++;
    +			} else if (path.endsWith(PENDING_SUFFIX)) {
    +				pend++;
    +			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
    +				val++;
    +			} else if (path.contains(PART_PREFIX)) {
    +				compl++;
    +			}
    +		}
    +
    +		Assert.assertEquals(inprogress, inProg);
    +		Assert.assertEquals(pending, pend);
    +		Assert.assertEquals(completed, compl);
    +		Assert.assertEquals(valid, val);
    +	}
    +
    +	static class ValidatingBucketingSink<T> extends BucketingSink<T> {
    +
    +		private static final long serialVersionUID = -4263974081712009141L;
    +
    +		public boolean initializeCalled = false;
    +
    +		ValidatingBucketingSink(String basePath) {
    +			super(basePath);
    +		}
    +
    +		/**
    +		 * The actual paths in this depend on the binary checkpoint so it you update this the paths
    +		 * here have to be updated as well.
    +		 */
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +			OperatorStateStore stateStore = context.getOperatorStateStore();
    +
    +			ListState<State<T>> restoredBucketStates = stateStore.getSerializableListState("bucket-states");
    +
    +			int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    --- End diff --
    
    Fixing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    @zentol Thanks again for your thorough review!
    
    What do you think about my earlier comment regarding unification:
    ```
    About unifying the functionality of the tests/ITCases, I was planning to to that for the upcoming *From13 tests but for the *From11 and *From12 it's difficult because quite some stuff changed between 1.1 and 1.2 and I want the code to mostly work on all branches. For example, this is the new *From12 tests rebased on the master branch: https://github.com/aljoscha/flink/tree/jira-5969-backwards-compat-12-13. By transplanting but keeping the snapshots that were generated on Flink 1.2.0 we get backwards compatibility tests while the tests in this PR test compatibility within Flink 1.2.x.
    
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114286782
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java ---
    @@ -208,11 +190,37 @@ protected final void restoreAndExecute(
     		JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph);
     
     		StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration());
    +		JobListeningContext jobListeningContext = clusterClient.connectToJob(jobSubmissionResult.getJobID());
     
     		boolean done = false;
     		while (DEADLINE.hasTimeLeft()) {
    +
    +			// try and get a job result, this will fail if the job already failed. Use this
    +			// to get out of this loop
    +			JobID jobId = jobSubmissionResult.getJobID();
    +			FiniteDuration timeout = FiniteDuration.apply(5, TimeUnit.SECONDS);
    +
    +			try {
    +
    +				Future<Object> future = Patterns.ask(clusterClient.getJobManagerGateway().actor(),
    --- End diff --
    
    Could use `ActorGateway#ask()` instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114342461
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedState.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedStateInOperator.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +		}
    +	}
    +
    +	public static class CheckingKeyedStateFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingKeyedStateFlatMap.class + "_RESTORE_CHECK";
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +	}
    +
    +	public static class CheckpointedUdfOperator
    +			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +		private static final long serialVersionUID = 1L;
    +
    +		private static final String CHECKPOINTED_STRING = "Oh my, that's nice!";
    +
    +		public CheckpointedUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
    +			super(userFunction);
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +
    +		@Override
    +		public void snapshotState(
    +				FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
    +			super.snapshotState(out, checkpointId, timestamp);
    +
    +			DataOutputViewStreamWrapper streamWrapper = new DataOutputViewStreamWrapper(out);
    +
    +			streamWrapper.writeUTF(CHECKPOINTED_STRING);
    +			streamWrapper.flush();
    +		}
    +	}
    +
    +	public static class CheckingRestoringUdfOperator
    +			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringUdfOperator.class + "_RESTORE_CHECK";
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
    +			super(userFunction);
    +		}
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
    +
    +			assertEquals(CheckpointedUdfOperator.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +
    +		@Override
    +		public void restoreState(FSDataInputStream in) throws Exception {
    +			super.restoreState(in);
    +
    +			DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
    +
    +			restoredState = streamWrapper.readUTF();
    +		}
    +	}
    +
    +	public static class TimelyStatefulOperator
    +			extends AbstractStreamOperator<Tuple2<Long, Long>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
    +		private static final long serialVersionUID = 1L;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		private transient InternalTimerService<Long> timerService;
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			timerService = getInternalTimerService(
    +					"timer",
    +					LongSerializer.INSTANCE,
    +					this);
    +
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					element.getValue().f0,
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			state.update(element.getValue().f1);
    +
    +			timerService.registerEventTimeTimer(element.getValue().f0, timerService.currentWatermark() + 10);
    +			timerService.registerProcessingTimeTimer(element.getValue().f0, timerService.currentProcessingTime() + 30_000);
    +
    +			output.collect(element);
    +		}
    +
    +		@Override
    +		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
    +
    +		}
    +
    +		@Override
    +		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
    +
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +	}
    +
    +	public static class CheckingTimelyStatefulOperator
    +			extends AbstractStreamOperator<Tuple2<Long, Long>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PROCESS_CHECKS";
    +		public static final String SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_ET_CHECKS";
    +		public static final String SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PT_CHECKS";
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		private transient InternalTimerService<Long> timerService;
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			timerService = getInternalTimerService(
    +					"timer",
    +					LongSerializer.INSTANCE,
    +					this);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, new IntCounter());
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, new IntCounter());
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					element.getValue().f0,
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), element.getValue().f1);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR).add(1);
    +
    +			output.collect(element);
    +		}
    +
    +		@Override
    +		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					timer.getNamespace(),
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), timer.getNamespace());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					timer.getNamespace(),
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), timer.getNamespace());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR).add(1);
    +		}
    +	}
    +
    +
    --- End diff --
    
    Fixing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113441899
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java ---
    @@ -155,12 +155,14 @@ public void snapshotState(FSDataOutputStream out, long checkpointId, long timest
     
     	@Override
     	public void restoreState(FSDataInputStream in) throws Exception {
    +		boolean readFlag = false;
     		if (userFunction instanceof Checkpointed ||
    -				(userFunction instanceof CheckpointedRestoring && in instanceof Migration)) {
    +				(userFunction instanceof CheckpointedRestoring)) {
    --- End diff --
    
    remove braces around `userFunction instanceof CheckpointedRestoring`; could also move it to the previous line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114344959
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    --- End diff --
    
    Changing to
    ```
    if (!latch.isTriggered()) {
    			latch.await(2, TimeUnit.MINUTES);
    		}
    
    		assertTrue("Latch was not triggered within the given timeout.", latch.isTriggered());
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114298622
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
    +		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();
    +		}
    +
    +		consumerOperator.close();
    +
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    if it isn't null we should print the error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114342062
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java ---
    @@ -485,7 +501,6 @@ public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws E
     	 * the operator implements this interface.
     	 */
     	@Deprecated
    -	@SuppressWarnings("deprecation")
    --- End diff --
    
    Seems to be a mistake. Fixing ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114298601
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    if it isn't null we should print the error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114293581
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMapWithKeyedState extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedState.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMapWithKeyedStateInOperator extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMapWithKeyedStateInOperator.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class KeyedStateSettingFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +		}
    +	}
    +
    +	public static class CheckingKeyedStateFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingKeyedStateFlatMap.class + "_RESTORE_CHECK";
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			ValueState<Long> state = getRuntimeContext().getState(stateDescriptor);
    +			if (state == null) {
    +				throw new RuntimeException("Missing key value state for " + value);
    +			}
    +
    +			assertEquals(value.f1, state.value());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +	}
    +
    +	public static class CheckpointedUdfOperator
    +			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +		private static final long serialVersionUID = 1L;
    +
    +		private static final String CHECKPOINTED_STRING = "Oh my, that's nice!";
    +
    +		public CheckpointedUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
    +			super(userFunction);
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +
    +		@Override
    +		public void snapshotState(
    +				FSDataOutputStream out, long checkpointId, long timestamp) throws Exception {
    +			super.snapshotState(out, checkpointId, timestamp);
    +
    +			DataOutputViewStreamWrapper streamWrapper = new DataOutputViewStreamWrapper(out);
    +
    +			streamWrapper.writeUTF(CHECKPOINTED_STRING);
    +			streamWrapper.flush();
    +		}
    +	}
    +
    +	public static class CheckingRestoringUdfOperator
    +			extends AbstractUdfStreamOperator<Tuple2<Long, Long>, FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringUdfOperator.class + "_RESTORE_CHECK";
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringUdfOperator(FlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>> userFunction) {
    +			super(userFunction);
    +		}
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			userFunction.flatMap(element.getValue(), new TimestampedCollector<>(output));
    +
    +			assertEquals(CheckpointedUdfOperator.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +
    +		@Override
    +		public void restoreState(FSDataInputStream in) throws Exception {
    +			super.restoreState(in);
    +
    +			DataInputViewStreamWrapper streamWrapper = new DataInputViewStreamWrapper(in);
    +
    +			restoredState = streamWrapper.readUTF();
    +		}
    +	}
    +
    +	public static class TimelyStatefulOperator
    +			extends AbstractStreamOperator<Tuple2<Long, Long>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
    +		private static final long serialVersionUID = 1L;
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		private transient InternalTimerService<Long> timerService;
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			timerService = getInternalTimerService(
    +					"timer",
    +					LongSerializer.INSTANCE,
    +					this);
    +
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					element.getValue().f0,
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			state.update(element.getValue().f1);
    +
    +			timerService.registerEventTimeTimer(element.getValue().f0, timerService.currentWatermark() + 10);
    +			timerService.registerProcessingTimeTimer(element.getValue().f0, timerService.currentProcessingTime() + 30_000);
    +
    +			output.collect(element);
    +		}
    +
    +		@Override
    +		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
    +
    +		}
    +
    +		@Override
    +		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
    +
    +		}
    +
    +		@Override
    +		public void processWatermark(Watermark mark) throws Exception {
    +			output.emitWatermark(mark);
    +		}
    +	}
    +
    +	public static class CheckingTimelyStatefulOperator
    +			extends AbstractStreamOperator<Tuple2<Long, Long>>
    +			implements OneInputStreamOperator<Tuple2<Long, Long>, Tuple2<Long, Long>>, Triggerable<Long, Long> {
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PROCESS_CHECKS";
    +		public static final String SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_ET_CHECKS";
    +		public static final String SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR = CheckingTimelyStatefulOperator.class + "_PT_CHECKS";
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		private transient InternalTimerService<Long> timerService;
    +
    +		@Override
    +		public void open() throws Exception {
    +			super.open();
    +
    +			timerService = getInternalTimerService(
    +					"timer",
    +					LongSerializer.INSTANCE,
    +					this);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, new IntCounter());
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, new IntCounter());
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void processElement(StreamRecord<Tuple2<Long, Long>> element) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					element.getValue().f0,
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), element.getValue().f1);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR).add(1);
    +
    +			output.collect(element);
    +		}
    +
    +		@Override
    +		public void onEventTime(InternalTimer<Long, Long> timer) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					timer.getNamespace(),
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), timer.getNamespace());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR).add(1);
    +		}
    +
    +		@Override
    +		public void onProcessingTime(InternalTimer<Long, Long> timer) throws Exception {
    +			ValueState<Long> state = getKeyedStateBackend().getPartitionedState(
    +					timer.getNamespace(),
    +					LongSerializer.INSTANCE,
    +					stateDescriptor);
    +
    +			assertEquals(state.value(), timer.getNamespace());
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR).add(1);
    +		}
    +	}
    +
    +
    --- End diff --
    
    remove 2 empty lines


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114343707
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		// compare if the results contain what they should contain and also if
    +		// they are the same, as they should.
    +
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
    +	}
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeMonitoringSourceSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		File path = null;
    --- End diff --
    
    Also leftovers, removing...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114342974
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    --- End diff --
    
    Feel free to keep it, it doesn't hurt after all. Just wanted to make sure i wasn't missing anything.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114288279
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java ---
    @@ -485,7 +501,6 @@ public OperatorStateHandles snapshot(long checkpointId, long timestamp) throws E
     	 * the operator implements this interface.
     	 */
     	@Deprecated
    -	@SuppressWarnings("deprecation")
    --- End diff --
    
    Why did you remove this? (and shouldn't the same apply to `void restore(StreamStateHandle)`?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114345481
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
    +		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();
    +		}
    +
    +		consumerOperator.close();
    +
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    See above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    Alright, so we're good to merge?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114296017
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		// compare if the results contain what they should contain and also if
    +		// they are the same, as they should.
    +
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
    +	}
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeMonitoringSourceSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		File path = null;
    --- End diff --
    
    This variable isn't used outside the loop, only the `fileModTime` is. Can be removed along with the assignment.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114344492
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		// compare if the results contain what they should contain and also if
    +		// they are the same, as they should.
    +
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
    +	}
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeMonitoringSourceSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		File path = null;
    +		long fileModTime = Long.MIN_VALUE;
    +		for (int i = 0; i < 1; i++) {
    +			Tuple2<File, String> file = createFileAndFillWithData(testFolder.getAbsolutePath(), "file", i, "This is test line.");
    +			path = file.f0;
    +			fileModTime = path.lastModified();
    +		}
    +
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +				new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.open();
    +
    +		final Throwable[] error = new Throwable[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		// run the source asynchronously
    +		Thread runner = new Thread() {
    +			@Override
    +			public void run() {
    +				try {
    +					monitoringFunction.run(new DummySourceContext() {
    +						@Override
    +						public void collect(TimestampedFileInputSplit element) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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,
    +				"src/test/resources/monitoring-function-migration-test-" + fileModTime +"-flink1.2-snapshot");
    +
    +		monitoringFunction.cancel();
    +		runner.join();
    +
    +		testHarness.close();
    +	}
    +
    +	@Test
    +	public void testMonitoringSourceRestore() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		Long expectedModTime = Long.parseLong("1493116191000");
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"monitoring-function-migration-test-1493116191000-flink1.2-snapshot"));
    +
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
    +
    +	}
    +
    +	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
    +
    +		private static final long serialVersionUID = -6727603565381560267L;
    +
    +		private final OneShotLatch latch;
    +
    +		private FileInputSplit split;
    +
    +		private boolean reachedEnd;
    +
    +		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
    +			super(filePath);
    +			this.latch = latch;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public void open(FileInputSplit fileSplit) throws IOException {
    +			this.split = fileSplit;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public boolean reachedEnd() throws IOException {
    +			if (!latch.isTriggered()) {
    +				try {
    +					latch.await();
    +				} catch (InterruptedException e) {
    +					e.printStackTrace();
    +				}
    +			}
    +			return reachedEnd;
    +		}
    +
    +		@Override
    +		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
    +			this.reachedEnd = true;
    +			return split;
    +		}
    +
    +		@Override
    +		public void close() {
    +
    +		}
    +	}
    +
    +	private static abstract class DummySourceContext
    +		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
    +
    +		private final Object lock = new Object();
    +
    +		@Override
    +		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
    +		}
    +
    +		@Override
    +		public void emitWatermark(Watermark mark) {
    +		}
    +
    +		@Override
    +		public Object getCheckpointLock() {
    +			return lock;
    +		}
    +
    +		@Override
    +		public void close() {
    +		}
    +	}
    +
    +	/**
    +	 * Create a file with pre-determined String format of the form:
    +	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
    +	 * */
    +	private Tuple2<File, String> createFileAndFillWithData(
    +		String base, String fileName, int fileIdx, String sampleLine) throws IOException {
    --- End diff --
    
    Fixing, what's up with the `[.]`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114347034
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		// compare if the results contain what they should contain and also if
    +		// they are the same, as they should.
    +
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
    +	}
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeMonitoringSourceSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		File path = null;
    +		long fileModTime = Long.MIN_VALUE;
    +		for (int i = 0; i < 1; i++) {
    +			Tuple2<File, String> file = createFileAndFillWithData(testFolder.getAbsolutePath(), "file", i, "This is test line.");
    +			path = file.f0;
    +			fileModTime = path.lastModified();
    +		}
    +
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +				new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.open();
    +
    +		final Throwable[] error = new Throwable[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		// run the source asynchronously
    +		Thread runner = new Thread() {
    +			@Override
    +			public void run() {
    +				try {
    +					monitoringFunction.run(new DummySourceContext() {
    +						@Override
    +						public void collect(TimestampedFileInputSplit element) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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,
    +				"src/test/resources/monitoring-function-migration-test-" + fileModTime +"-flink1.2-snapshot");
    +
    +		monitoringFunction.cancel();
    +		runner.join();
    +
    +		testHarness.close();
    +	}
    +
    +	@Test
    +	public void testMonitoringSourceRestore() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		Long expectedModTime = Long.parseLong("1493116191000");
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"monitoring-function-migration-test-1493116191000-flink1.2-snapshot"));
    +
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
    +
    +	}
    +
    +	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
    +
    +		private static final long serialVersionUID = -6727603565381560267L;
    +
    +		private final OneShotLatch latch;
    +
    +		private FileInputSplit split;
    +
    +		private boolean reachedEnd;
    +
    +		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
    +			super(filePath);
    +			this.latch = latch;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public void open(FileInputSplit fileSplit) throws IOException {
    +			this.split = fileSplit;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public boolean reachedEnd() throws IOException {
    +			if (!latch.isTriggered()) {
    +				try {
    +					latch.await();
    +				} catch (InterruptedException e) {
    +					e.printStackTrace();
    +				}
    +			}
    +			return reachedEnd;
    +		}
    +
    +		@Override
    +		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
    +			this.reachedEnd = true;
    +			return split;
    +		}
    +
    +		@Override
    +		public void close() {
    +
    +		}
    +	}
    +
    +	private static abstract class DummySourceContext
    +		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
    +
    +		private final Object lock = new Object();
    +
    +		@Override
    +		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
    +		}
    +
    +		@Override
    +		public void emitWatermark(Watermark mark) {
    +		}
    +
    +		@Override
    +		public Object getCheckpointLock() {
    +			return lock;
    +		}
    +
    +		@Override
    +		public void close() {
    +		}
    +	}
    +
    +	/**
    +	 * Create a file with pre-determined String format of the form:
    +	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
    +	 * */
    +	private Tuple2<File, String> createFileAndFillWithData(
    +		String base, String fileName, int fileIdx, String sampleLine) throws IOException {
    --- End diff --
    
    the line below has a `.` before fileName, (tmp variable) and I was trying to cover both cases with 1 comment by including `[.]` as some kind of optional part of the name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113435697
  
    --- Diff: flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java ---
    @@ -489,6 +489,39 @@ public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException
     	}
     
     	/**
    +	 * Reattaches to a running from from the supplied job id
    +	 *
    +	 * @param jobID The job id of the job to attach to
    +	 * @return The JobExecutionResult for the jobID
    +	 * @throws JobExecutionException if an error occurs during monitoring the job execution
    +	 */
    +	public JobListeningContext connectToJob(JobID jobID) throws JobExecutionException {
    +		final LeaderRetrievalService leaderRetrievalService;
    +		try {
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
    +		} catch (Exception e) {
    +			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
    +		}
    +
    +		ActorGateway jobManagerGateway;
    +		try {
    +			jobManagerGateway = getJobManagerGateway();
    +		} catch (Exception e) {
    +			throw new JobRetrievalException(jobID, "Could not retrieve the JobManager Gateway");
    --- End diff --
    
    Include the exception


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha closed the pull request at:

    https://github.com/apache/flink/pull/3778


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113435588
  
    --- Diff: flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java ---
    @@ -489,6 +489,39 @@ public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException
     	}
     
     	/**
    +	 * Reattaches to a running from from the supplied job id
    --- End diff --
    
    typo: from -> job


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114301414
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    --- End diff --
    
    Can we use the partitions stored in `state1` for this? Just so we don't define the same partitions in multiple places.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114298586
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    if it isn't null we should print the error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114342231
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    --- End diff --
    
    There was a point in debugging this whole thing were I wasn't sure wether state was set correctly, so I introduced this sanity check. Could remove it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114294138
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulJobSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    --- End diff --
    
    Isn't this always true? We just set the value after all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114345125
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    Changing to
    ```
    		assertNull("Got error: " + error[0], error[0]);
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    One thing i was wondering about regarding the added IT cases is whether we can create a single class with the logic for both 1.1 and 1.2. Given that the new IT cases appear to be mostly a copy of the existing tests, shouldn't the only difference be which savepoint we use?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114301599
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    --- End diff --
    
    Maybe even throw them into a static field that all tests use.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114344618
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    --- End diff --
    
    But when does the Reader get unblocked? Or do we just destroy it somewhere?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    Yep, though it is code duplication both across time (releases) and versions, now. 😅 I definitely have plans to unify for the `*From13MigrationTests`, though, since the interface is a bit more stable starting with 1.2.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114304332
  
    --- Diff: flink-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSinkFrom12MigrationTest.java ---
    @@ -0,0 +1,224 @@
    +/*
    + * 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.fs.bucketing;
    +
    +import static org.junit.Assert.assertTrue;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.util.List;
    +import java.util.Map;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.OperatorStateStore;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.connectors.fs.StringWriter;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.hadoop.fs.Path;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +/**
    + * Tests for checking whether {@link BucketingSink} can restore from snapshots that were done
    + * using the Flink 1.2 {@link BucketingSink}.
    + *
    + * <p>For regenerating the binary snapshot file you have to run the {@code write*()} method on
    + * the Flink 1.2 branch.
    + */
    +
    +public class BucketingSinkFrom12MigrationTest {
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	private static final String PART_PREFIX = "part";
    +	private static final String PENDING_SUFFIX = ".pending";
    +	private static final String IN_PROGRESS_SUFFIX = ".in-progress";
    +	private static final String VALID_LENGTH_SUFFIX = ".valid";
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +
    +		final File outDir = tempFolder.newFolder();
    +
    +		BucketingSink<String> sink = new BucketingSink<String>(outDir.getAbsolutePath())
    +			.setWriter(new StringWriter<String>())
    +			.setBatchSize(5)
    +			.setPartPrefix(PART_PREFIX)
    +			.setInProgressPrefix("")
    +			.setPendingPrefix("")
    +			.setValidLengthPrefix("")
    +			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
    +			.setPendingSuffix(PENDING_SUFFIX)
    +			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
    +
    +		OneInputStreamOperatorTestHarness<String, Object> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink));
    +
    +		testHarness.setup();
    +		testHarness.open();
    +
    +		testHarness.processElement(new StreamRecord<>("test1", 0L));
    +		testHarness.processElement(new StreamRecord<>("test2", 0L));
    +
    +		checkFs(outDir, 1, 1, 0, 0);
    +
    +		testHarness.processElement(new StreamRecord<>("test3", 0L));
    +		testHarness.processElement(new StreamRecord<>("test4", 0L));
    +		testHarness.processElement(new StreamRecord<>("test5", 0L));
    +
    +		checkFs(outDir, 1, 4, 0, 0);
    +
    +		OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L);
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/bucketing-sink-migration-test-flink1.2-snapshot");
    +		testHarness.close();
    +	}
    +
    +	@Test
    +	public void testRestore() throws Exception {
    +		final File outDir = tempFolder.newFolder();
    +
    +		ValidatingBucketingSink<String> sink = (ValidatingBucketingSink<String>) new ValidatingBucketingSink<String>(outDir.getAbsolutePath())
    +			.setWriter(new StringWriter<String>())
    +			.setBatchSize(5)
    +			.setPartPrefix(PART_PREFIX)
    +			.setInProgressPrefix("")
    +			.setPendingPrefix("")
    +			.setValidLengthPrefix("")
    +			.setInProgressSuffix(IN_PROGRESS_SUFFIX)
    +			.setPendingSuffix(PENDING_SUFFIX)
    +			.setValidLengthSuffix(VALID_LENGTH_SUFFIX);
    +
    +		OneInputStreamOperatorTestHarness<String, Object> testHarness = new OneInputStreamOperatorTestHarness<>(
    +			new StreamSink<>(sink), 10, 1, 0);
    +		testHarness.setup();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("bucketing-sink-migration-test-flink1.2-snapshot")));
    +		testHarness.open();
    +
    +		assertTrue(sink.initializeCalled);
    +
    +		testHarness.processElement(new StreamRecord<>("test1", 0L));
    +		testHarness.processElement(new StreamRecord<>("test2", 0L));
    +
    +		checkFs(outDir, 1, 1, 0, 0);
    +
    +		testHarness.close();
    +	}
    +
    +	private void checkFs(File outDir, int inprogress, int pending, int completed, int valid) throws IOException {
    +		int inProg = 0;
    +		int pend = 0;
    +		int compl = 0;
    +		int val = 0;
    +
    +		for (File file: FileUtils.listFiles(outDir, null, true)) {
    +			if (file.getAbsolutePath().endsWith("crc")) {
    +				continue;
    +			}
    +			String path = file.getPath();
    +			if (path.endsWith(IN_PROGRESS_SUFFIX)) {
    +				inProg++;
    +			} else if (path.endsWith(PENDING_SUFFIX)) {
    +				pend++;
    +			} else if (path.endsWith(VALID_LENGTH_SUFFIX)) {
    +				val++;
    +			} else if (path.contains(PART_PREFIX)) {
    +				compl++;
    +			}
    +		}
    +
    +		Assert.assertEquals(inprogress, inProg);
    +		Assert.assertEquals(pending, pend);
    +		Assert.assertEquals(completed, compl);
    +		Assert.assertEquals(valid, val);
    +	}
    +
    +	static class ValidatingBucketingSink<T> extends BucketingSink<T> {
    +
    +		private static final long serialVersionUID = -4263974081712009141L;
    +
    +		public boolean initializeCalled = false;
    +
    +		ValidatingBucketingSink(String basePath) {
    +			super(basePath);
    +		}
    +
    +		/**
    +		 * The actual paths in this depend on the binary checkpoint so it you update this the paths
    +		 * here have to be updated as well.
    +		 */
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +			OperatorStateStore stateStore = context.getOperatorStateStore();
    +
    +			ListState<State<T>> restoredBucketStates = stateStore.getSerializableListState("bucket-states");
    +
    +			int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    --- End diff --
    
    unused variable


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114297029
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		// compare if the results contain what they should contain and also if
    +		// they are the same, as they should.
    +
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split1)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split2)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split3)));
    +		Assert.assertTrue(testHarness.getOutput().contains(new StreamRecord<>(split4)));
    +	}
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeMonitoringSourceSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		File path = null;
    +		long fileModTime = Long.MIN_VALUE;
    +		for (int i = 0; i < 1; i++) {
    +			Tuple2<File, String> file = createFileAndFillWithData(testFolder.getAbsolutePath(), "file", i, "This is test line.");
    +			path = file.f0;
    +			fileModTime = path.lastModified();
    +		}
    +
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +				new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.open();
    +
    +		final Throwable[] error = new Throwable[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		// run the source asynchronously
    +		Thread runner = new Thread() {
    +			@Override
    +			public void run() {
    +				try {
    +					monitoringFunction.run(new DummySourceContext() {
    +						@Override
    +						public void collect(TimestampedFileInputSplit element) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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,
    +				"src/test/resources/monitoring-function-migration-test-" + fileModTime +"-flink1.2-snapshot");
    +
    +		monitoringFunction.cancel();
    +		runner.join();
    +
    +		testHarness.close();
    +	}
    +
    +	@Test
    +	public void testMonitoringSourceRestore() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		Long expectedModTime = Long.parseLong("1493116191000");
    +		TextInputFormat format = new TextInputFormat(new Path(testFolder.getAbsolutePath()));
    +
    +		final ContinuousFileMonitoringFunction<String> monitoringFunction =
    +			new ContinuousFileMonitoringFunction<>(format, FileProcessingMode.PROCESS_CONTINUOUSLY, 1, INTERVAL);
    +
    +		StreamSource<TimestampedFileInputSplit, ContinuousFileMonitoringFunction<String>> src =
    +			new StreamSource<>(monitoringFunction);
    +
    +		final AbstractStreamOperatorTestHarness<TimestampedFileInputSplit> testHarness =
    +			new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"monitoring-function-migration-test-1493116191000-flink1.2-snapshot"));
    +
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		Assert.assertEquals((long) expectedModTime, monitoringFunction.getGlobalModificationTime());
    +
    +	}
    +
    +	private static class BlockingFileInputFormat extends FileInputFormat<FileInputSplit> {
    +
    +		private static final long serialVersionUID = -6727603565381560267L;
    +
    +		private final OneShotLatch latch;
    +
    +		private FileInputSplit split;
    +
    +		private boolean reachedEnd;
    +
    +		BlockingFileInputFormat(OneShotLatch latch, Path filePath) {
    +			super(filePath);
    +			this.latch = latch;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public void open(FileInputSplit fileSplit) throws IOException {
    +			this.split = fileSplit;
    +			this.reachedEnd = false;
    +		}
    +
    +		@Override
    +		public boolean reachedEnd() throws IOException {
    +			if (!latch.isTriggered()) {
    +				try {
    +					latch.await();
    +				} catch (InterruptedException e) {
    +					e.printStackTrace();
    +				}
    +			}
    +			return reachedEnd;
    +		}
    +
    +		@Override
    +		public FileInputSplit nextRecord(FileInputSplit reuse) throws IOException {
    +			this.reachedEnd = true;
    +			return split;
    +		}
    +
    +		@Override
    +		public void close() {
    +
    +		}
    +	}
    +
    +	private static abstract class DummySourceContext
    +		implements SourceFunction.SourceContext<TimestampedFileInputSplit> {
    +
    +		private final Object lock = new Object();
    +
    +		@Override
    +		public void collectWithTimestamp(TimestampedFileInputSplit element, long timestamp) {
    +		}
    +
    +		@Override
    +		public void emitWatermark(Watermark mark) {
    +		}
    +
    +		@Override
    +		public Object getCheckpointLock() {
    +			return lock;
    +		}
    +
    +		@Override
    +		public void close() {
    +		}
    +	}
    +
    +	/**
    +	 * Create a file with pre-determined String format of the form:
    +	 * {@code fileIdx +": "+ sampleLine +" "+ lineNo}.
    +	 * */
    +	private Tuple2<File, String> createFileAndFillWithData(
    +		String base, String fileName, int fileIdx, String sampleLine) throws IOException {
    --- End diff --
    
    We could pass the base as a `File`, then you could create the new file like this: ```new File(base, [.]fileName + fileIdx)```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    Manually merged


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    yes, +1 from my side.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    Thanks @zentol! I addressed your (easily resolvable) comments.
    
    About unifying the functionality of the tests/ITCases, I was planning to to that for the upcoming `*From13` tests but for the `*From11` and `*From12` it's difficult because quite some stuff changed between 1.1 and 1.2 and I want the code to mostly work on all branches. For example, this is the new `*From12` tests rebased on the master branch: https://github.com/aljoscha/flink/tree/jira-5969-backwards-compat-12-13. By transplanting but keeping the snapshots that were generated on Flink 1.2.0 we get backwards compatibility tests while the tests in this PR test compatibility within Flink 1.2.x. 
    
    I'm hoping that for the future we can move to a base class and version specific tests. I'll also think about how much is possible right now and try and simplify/unify.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    @zentol Thanks for reviewing! 😃 
    
    I merged this also on master, so that we have backwards compatibility tests from 1.2 to the current master, which was the reason for this whole exercise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3778: [FLINK-5969] Add savepoint backwards compatibility tests ...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/3778
  
    @aljoscha Having the same code on all branches is rather nice; never looked at it from that angle before. Not modifying existing tests is a plus anyway; so trying to unify from here on out sounds like a good idea.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113435639
  
    --- Diff: flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java ---
    @@ -489,6 +489,39 @@ public JobExecutionResult retrieveJob(JobID jobID) throws JobExecutionException
     	}
     
     	/**
    +	 * Reattaches to a running from from the supplied job id
    +	 *
    +	 * @param jobID The job id of the job to attach to
    +	 * @return The JobExecutionResult for the jobID
    +	 * @throws JobExecutionException if an error occurs during monitoring the job execution
    +	 */
    +	public JobListeningContext connectToJob(JobID jobID) throws JobExecutionException {
    +		final LeaderRetrievalService leaderRetrievalService;
    +		try {
    +			leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(flinkConfig);
    +		} catch (Exception e) {
    +			throw new JobRetrievalException(jobID, "Could not create the leader retrieval service", e);
    +		}
    +
    +		ActorGateway jobManagerGateway;
    +		try {
    +			jobManagerGateway = getJobManagerGateway();
    +		} catch (Exception e) {
    +			throw new JobRetrievalException(jobID, "Could not retrieve the JobManager Gateway");
    +		}
    +
    +		return JobClient.attachToRunningJob(
    +				jobID,
    +				jobManagerGateway,
    +				flinkConfig,
    +				actorSystemLoader.get(),
    +				leaderRetrievalService,
    +				timeout,
    +				printStatusDuringExecution);
    +	}
    +
    +
    --- End diff --
    
    remove empty line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113440360
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(
    +				format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +				new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +		testHarness.open();
    +		// create some state in the reader
    +		testHarness.processElement(new StreamRecord<>(split1));
    +		testHarness.processElement(new StreamRecord<>(split2));
    +		testHarness.processElement(new StreamRecord<>(split3));
    +		testHarness.processElement(new StreamRecord<>(split4));
    +		// take a snapshot of the operator's state. This will be used
    +		// to initialize another reader and compare the results of the
    +		// two operators.
    +
    +		final OperatorStateHandles snapshot;
    +		synchronized (testHarness.getCheckpointLock()) {
    +			snapshot = testHarness.snapshot(0L, 0L);
    +		}
    +
    +		OperatorSnapshotUtil.writeStateHandle(snapshot, "src/test/resources/reader-migration-test-flink1.2-snapshot");
    +	}
    +
    +	@Test
    +	public void testReaderRestore() throws Exception {
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +			new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +			new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +			new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +			new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +
    +		BlockingFileInputFormat format = new BlockingFileInputFormat(latch, new Path(testFolder.getAbsolutePath()));
    +		TypeInformation<FileInputSplit> typeInfo = TypeExtractor.getInputFormatTypes(format);
    +
    +		ContinuousFileReaderOperator<FileInputSplit> initReader = new ContinuousFileReaderOperator<>(format);
    +		initReader.setOutputType(typeInfo, new ExecutionConfig());
    +
    +		OneInputStreamOperatorTestHarness<TimestampedFileInputSplit, FileInputSplit> testHarness =
    +			new OneInputStreamOperatorTestHarness<>(initReader);
    +		testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime);
    +
    +		testHarness.setup();
    +		OperatorStateHandles operatorStateHandles = OperatorSnapshotUtil.readStateHandle(
    +				OperatorSnapshotUtil.getResourceFilename(
    +						"reader-migration-test-flink1.2-snapshot"));
    +		testHarness.initializeState(operatorStateHandles);
    +		testHarness.open();
    +
    +		latch.trigger();
    +
    +		// ... and wait for the operators to close gracefully
    +
    +		synchronized (testHarness.getCheckpointLock()) {
    +			testHarness.close();
    +		}
    +
    +		FileInputSplit fsSplit1 = createSplitFromTimestampedSplit(split1);
    --- End diff --
    
    you could create the FileInputSplits directly here, remove the TimestampedFileInputSplit instantiation at the start of this method and then remove the helper function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114300274
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    --- End diff --
    
    What do you think about adding a timeout here to be safe? (and of course other calls to this method in this file)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114300783
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
    +		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();
    --- End diff --
    
    remove this line?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114345452
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    --- End diff --
    
    See above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114353314
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    --- End diff --
    
    Replacing by
    ```
    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);
    }
    ```
     and
    ```
    final List<KafkaTopicPartition> partitions = new ArrayList<>(PARTITION_STATE.keySet());
    ```
    when creating the partitions list


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114343238
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    --- End diff --
    
    Yes, it's intended, to ensure that the Reader never actually does any reading. I copied this straight from the old tests but I agree that it's not very apparent. I'll add a comment 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113442453
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java ---
    @@ -155,12 +155,14 @@ public void snapshotState(FSDataOutputStream out, long checkpointId, long timest
     
     	@Override
     	public void restoreState(FSDataInputStream in) throws Exception {
    +		boolean readFlag = false;
    --- End diff --
    
    the variable name is a bit ambiguous, (it could also mean "you have to read the flag".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r113446953
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulUDFSavepointFrom12MigrationITCase.java ---
    @@ -0,0 +1,771 @@
    +/*
    + * 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.test.checkpointing.utils;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.flink.api.common.accumulators.IntCounter;
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.core.fs.FSDataInputStream;
    +import org.apache.flink.core.fs.FSDataOutputStream;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.checkpoint.Checkpointed;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
    +import org.apache.flink.streaming.api.operators.InternalTimer;
    +import org.apache.flink.streaming.api.operators.InternalTimerService;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.operators.TimestampedCollector;
    +import org.apache.flink.streaming.api.operators.Triggerable;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.util.Collector;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +
    +/**
    + * This verifies that we can restore a complete job from a Flink 1.2 savepoint.
    + *
    + * <p>The test pipeline contains both "Checkpointed" state and keyed user state.
    + *
    + * <p>The tests will time out if they don't see the required number of successful checks within
    + * a time limit.
    + */
    +public class StatefulUDFSavepointFrom12MigrationITCase extends SavepointMigrationTestBase {
    +	private static final int NUM_SOURCE_ELEMENTS = 4;
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	/**
    +	 * This has to be manually executed to create the savepoint on Flink 1.2.
    +	 */
    +	@Test
    +	@Ignore
    +	public void testCreateSavepointOnFlink12WithRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		RocksDBStateBackend rocksBackend =
    +				new RocksDBStateBackend(new MemoryStateBackend());
    +		env.setStateBackend(rocksBackend);
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new LegacyCheckpointedSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new LegacyCheckpointedFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new LegacyCheckpointedFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new KeyedStateSettingFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckpointedUdfOperator(new LegacyCheckpointedFlatMapWithKeyedState())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new TimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		executeAndSavepoint(
    +				env,
    +				"src/test/resources/stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb",
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new MemoryStateBackend());
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	@Test
    +	public void testSavepointRestoreFromFlink12FromRocksDB() throws Exception {
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +		env.setRestartStrategy(RestartStrategies.noRestart());
    +		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
    +		// we only test memory state backend yet
    +		env.setStateBackend(new RocksDBStateBackend(new MemoryStateBackend()));
    +		env.enableCheckpointing(500);
    +		env.setParallelism(4);
    +		env.setMaxParallelism(4);
    +
    +		env
    +				.addSource(new CheckingRestoringSource(NUM_SOURCE_ELEMENTS)).setMaxParallelism(1).uid("LegacyCheckpointedSource")
    +				.flatMap(new CheckingRestoringFlatMap()).startNewChain().uid("LegacyCheckpointedFlatMap")
    +				.keyBy(0)
    +				.flatMap(new CheckingRestoringFlatMapWithKeyedState()).startNewChain().uid("LegacyCheckpointedFlatMapWithKeyedState")
    +				.keyBy(0)
    +				.flatMap(new CheckingKeyedStateFlatMap()).startNewChain().uid("KeyedStateSettingFlatMap")
    +				.keyBy(0)
    +				.transform(
    +						"custom_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingRestoringUdfOperator(new CheckingRestoringFlatMapWithKeyedStateInOperator())).uid("LegacyCheckpointedOperator")
    +				.keyBy(0)
    +				.transform(
    +						"timely_stateful_operator",
    +						new TypeHint<Tuple2<Long, Long>>() {}.getTypeInfo(),
    +						new CheckingTimelyStatefulOperator()).uid("TimelyStatefulOperator")
    +				.addSink(new AccumulatorCountingSink<Tuple2<Long, Long>>());
    +
    +		restoreAndExecute(
    +				env,
    +				getResourceFilename("stateful-udf-migration-itcase-flink1.2-savepoint-rocksdb"),
    +				new Tuple2<>(CheckingRestoringSource.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, 1),
    +				new Tuple2<>(CheckingRestoringFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedState.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingKeyedStateFlatMap.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringUdfOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingRestoringFlatMapWithKeyedStateInOperator.SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESS_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_EVENT_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(CheckingTimelyStatefulOperator.SUCCESSFUL_PROCESSING_TIME_CHECK_ACCUMULATOR, NUM_SOURCE_ELEMENTS),
    +				new Tuple2<>(AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, NUM_SOURCE_ELEMENTS));
    +	}
    +
    +	private static class LegacyCheckpointedSource
    +			implements SourceFunction<Tuple2<Long, Long>>, Checkpointed<String> {
    +
    +		public static String CHECKPOINTED_STRING = "Here be dragons!";
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		public LegacyCheckpointedSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +
    +			ctx.emitWatermark(new Watermark(0));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			// don't emit a final watermark so that we don't trigger the registered event-time
    +			// timers
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			assertEquals(CHECKPOINTED_STRING, state);
    +		}
    +
    +		@Override
    +		public String snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_STRING;
    +		}
    +	}
    +
    +	private static class CheckingRestoringSource
    +			extends RichSourceFunction<Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<String> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringSource.class + "_RESTORE_CHECK";
    +
    +		private volatile boolean isRunning = true;
    +
    +		private final int numElements;
    +
    +		private String restoredState;
    +
    +		public CheckingRestoringSource(int numElements) {
    +			this.numElements = numElements;
    +		}
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Tuple2<Long, Long>> ctx) throws Exception {
    +			assertEquals(LegacyCheckpointedSource.CHECKPOINTED_STRING, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +			// immediately trigger any set timers
    +			ctx.emitWatermark(new Watermark(1000));
    +
    +			synchronized (ctx.getCheckpointLock()) {
    +				for (long i = 0; i < numElements; i++) {
    +					ctx.collect(new Tuple2<>(i, i));
    +				}
    +			}
    +
    +			while (isRunning) {
    +				Thread.sleep(20);
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			isRunning = false;
    +		}
    +
    +		@Override
    +		public void restoreState(String state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +		}
    +
    +		@Override
    +		public Tuple2<String, Long> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
    +			return CHECKPOINTED_TUPLE;
    +		}
    +	}
    +
    +	public static class CheckingRestoringFlatMap extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements CheckpointedRestoring<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static final String SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR = CheckingRestoringFlatMap.class + "_RESTORE_CHECK";
    +
    +		private transient Tuple2<String, Long> restoredState;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +
    +			getRuntimeContext().addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter());
    +		}
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			assertEquals(LegacyCheckpointedFlatMap.CHECKPOINTED_TUPLE, restoredState);
    +			getRuntimeContext().getAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR).add(1);
    +
    +		}
    +
    +		@Override
    +		public void restoreState(Tuple2<String, Long> state) throws Exception {
    +			restoredState = state;
    +		}
    +	}
    +
    +	public static class LegacyCheckpointedFlatMapWithKeyedState
    +			extends RichFlatMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>>
    +			implements Checkpointed<Tuple2<String, Long>> {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		public static Tuple2<String, Long> CHECKPOINTED_TUPLE =
    +				new Tuple2<>("hello", 42L);
    +
    +		private final ValueStateDescriptor<Long> stateDescriptor =
    +				new ValueStateDescriptor<Long>("state-name", LongSerializer.INSTANCE);
    +
    +		@Override
    +		public void flatMap(Tuple2<Long, Long> value, Collector<Tuple2<Long, Long>> out) throws Exception {
    +			out.collect(value);
    +
    +			getRuntimeContext().getState(stateDescriptor).update(value.f1);
    +
    +			assertEquals(value.f1, getRuntimeContext().getState(stateDescriptor).value());
    --- End diff --
    
    Isn't this always true? We just set the value after all.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114353390
  
    --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseFrom12MigrationTest.java ---
    @@ -0,0 +1,491 @@
    +/*
    + * 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.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Matchers.anyMapOf;
    +import static org.mockito.Mockito.doAnswer;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +import java.io.Serializable;
    +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.internals.AbstractFetcher;
    +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
    +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 {
    +
    +
    +	/**
    +	 * Manually run this to write binary snapshot data.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeSnapshot() throws Exception {
    +		final HashMap<KafkaTopicPartition, Long> state = new HashMap<>();
    +		state.put(new KafkaTopicPartition("abc", 13), 16768L);
    +		state.put(new KafkaTopicPartition("def", 7), 987654321L);
    +		writeSnapshot("src/test/resources/kafka-consumer-migration-test-flink1.2-snapshot", 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<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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) {
    +							latch.trigger();
    +						}
    +					});
    +				}
    +				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
    +	public void testRestoreWithEmptyStateNoPartitions() throws Exception {
    +		// --------------------------------------------------------------------
    +		//   prepare fake states
    +		// --------------------------------------------------------------------
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return fetcher;
    +					}
    +				},
    +				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();
    +		testHarness.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.cancel();
    +		consumerOperator.close();
    +
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestoreWithEmptyStateWithPartitions() throws Exception {
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				latch.trigger();
    +				Assert.fail("This should never be called");
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +		doAnswer(new Answer<Void>() {
    +			@Override
    +			public Void answer(InvocationOnMock invocation) throws Throwable {
    +				latch.trigger();
    +				return null;
    +			}
    +		}).when(fetcher).runFetchLoop();
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot-empty-state")));
    +		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();
    +							Assert.fail("This should never be called.");
    +						}
    +
    +						@Override
    +						public void emitWatermark(Watermark mark) {
    +							latch.trigger();
    +							assertEquals(Long.MAX_VALUE, mark.getTimestamp());
    +						}
    +					});
    +				}
    +				catch (Throwable t) {
    +					t.printStackTrace();
    +					error[0] = t;
    +				}
    +			}
    +		};
    +		runner.start();
    +
    +		if (!latch.isTriggered()) {
    +			latch.await();
    +		}
    +
    +		consumerOperator.close();
    +		runner.interrupt();
    +		runner.join();
    +
    +		assertNull(error[0]);
    +	}
    +
    +	@Test
    +	public void testRestore() 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 boolean[] verifiedState = new boolean[1];
    +
    +		final OneShotLatch latch = new OneShotLatch();
    +		final AbstractFetcher<String, ?> fetcher = mock(AbstractFetcher.class);
    +
    +		doAnswer(new Answer() {
    +			@Override
    +			public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
    +				Map<KafkaTopicPartition, Long> map = (HashMap<KafkaTopicPartition, Long>) invocationOnMock.getArguments()[0];
    +
    +				latch.trigger();
    +				assertEquals(state1, map);
    +				verifiedState[0] = true;
    +				return null;
    +			}
    +		}).when(fetcher).restoreOffsets(anyMapOf(KafkaTopicPartition.class, Long.class));
    +
    +
    +		final List<KafkaTopicPartition> partitions = new ArrayList<>();
    +		partitions.add(new KafkaTopicPartition("abc", 13));
    +		partitions.add(new KafkaTopicPartition("def", 7));
    +
    +		final DummyFlinkKafkaConsumer<String> consumerFunction = new DummyFlinkKafkaConsumer<>(
    +				new FetcherFactory<String>() {
    +					private static final long serialVersionUID = -2803131905656983619L;
    +
    +					@Override
    +					public AbstractFetcher<String, ?> createFetcher() {
    +						return 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.initializeState(
    +				OperatorSnapshotUtil.readStateHandle(
    +						OperatorSnapshotUtil.getResourceFilename("kafka-consumer-migration-test-flink1.2-snapshot")));
    +		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();
    --- End diff --
    
    Fixing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114342323
  
    --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java ---
    @@ -208,11 +190,37 @@ protected final void restoreAndExecute(
     		JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph);
     
     		StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration());
    +		JobListeningContext jobListeningContext = clusterClient.connectToJob(jobSubmissionResult.getJobID());
     
     		boolean done = false;
     		while (DEADLINE.hasTimeLeft()) {
    +
    +			// try and get a job result, this will fail if the job already failed. Use this
    +			// to get out of this loop
    +			JobID jobId = jobSubmissionResult.getJobID();
    +			FiniteDuration timeout = FiniteDuration.apply(5, TimeUnit.SECONDS);
    +
    +			try {
    +
    +				Future<Object> future = Patterns.ask(clusterClient.getJobManagerGateway().actor(),
    --- End diff --
    
    Changing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114297917
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    --- End diff --
    
    this latch is never triggered; intended?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3778: [FLINK-5969] Add savepoint backwards compatibility...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3778#discussion_r114356062
  
    --- Diff: flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingFrom12MigrationTest.java ---
    @@ -0,0 +1,360 @@
    +/*
    + * 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.hdfstests;
    +
    +import java.io.FileOutputStream;
    +import org.apache.commons.io.FileUtils;
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.api.common.io.FileInputFormat;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.io.TextInputFormat;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.core.fs.FileInputSplit;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.streaming.api.TimeCharacteristic;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileMonitoringFunction;
    +import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperator;
    +import org.apache.flink.streaming.api.functions.source.FileProcessingMode;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;
    +import org.apache.flink.streaming.api.operators.StreamSource;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
    +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
    +import org.apache.flink.streaming.util.OperatorSnapshotUtil;
    +import org.apache.flink.util.Preconditions;
    +import org.junit.Assert;
    +import org.junit.ClassRule;
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.junit.rules.TemporaryFolder;
    +
    +import java.io.File;
    +import java.io.IOException;
    +
    +public class ContinuousFileProcessingFrom12MigrationTest {
    +
    +	private static final int LINES_PER_FILE = 10;
    +
    +	private static final long INTERVAL = 100;
    +
    +	@ClassRule
    +	public static TemporaryFolder tempFolder = new TemporaryFolder();
    +
    +	/**
    +	 * Manually run this to write binary snapshot data. Remove @Ignore to run.
    +	 */
    +	@Ignore
    +	@Test
    +	public void writeReaderSnapshot() throws Exception {
    +
    +		File testFolder = tempFolder.newFolder();
    +
    +		TimestampedFileInputSplit split1 =
    +				new TimestampedFileInputSplit(0, 3, new Path("test/test1"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split2 =
    +				new TimestampedFileInputSplit(10, 2, new Path("test/test2"), 101, 200, null);
    +
    +		TimestampedFileInputSplit split3 =
    +				new TimestampedFileInputSplit(10, 1, new Path("test/test2"), 0, 100, null);
    +
    +		TimestampedFileInputSplit split4 =
    +				new TimestampedFileInputSplit(11, 0, new Path("test/test3"), 0, 100, null);
    +
    +		final OneShotLatch latch = new OneShotLatch();
    --- End diff --
    
    It never get's unblocked and just goes away when the test finishes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---