You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StefanRRichter <gi...@git.apache.org> on 2018/03/09 16:41:44 UTC

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

GitHub user StefanRRichter opened a pull request:

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

    [FLINK-8910][tests] Automated end-to-end test for local recovery and sticky scheduling

    This PR adds an automated end-to-end test for the local recovery feature, which also includes sticky allocation. We expose allocation id through (not public) `StreamingRuntimeContext `.


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

    $ git pull https://github.com/StefanRRichter/flink automated-test-finish

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

    https://github.com/apache/flink/pull/5676.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 #5676
    
----
commit 7de3d9d8ddaa84684e8c757285201387ad556ef2
Author: Stefan Richter <s....@...>
Date:   2018-03-08T18:20:32Z

    [FLINK-8910][tests] Expose allocation id through runtime ctx

commit d8e740484a340f20277b9ed1e2d22b9d96897937
Author: Stefan Richter <s....@...>
Date:   2018-03-06T09:35:44Z

    [FLINK-8910][tests] Automated test for local recovery (including sticky allocation)

----


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r176765707
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    AFAIK, the SO license is not compatible with the ASL. So if it comes from SO, we cannot use it.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175828193
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    --- End diff --
    
    same here


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r177468202
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    Stack Overflow is [licensed CC BY-SA 3.0](https://stackoverflow.com/help/licensing) which is [Apache approved](https://www.apache.org/legal/resolved.html#cc-sa). Our attributions do need to be more than a link.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178804294
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    Can we do the test without this information? It feels a bit as if we introduce this field only to realize the test.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175820558
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    --- End diff --
    
    Shall we put the job in its own module to simplify the jar building? This will also help to keep the dependencies of several jobs separated.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175826404
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    --- End diff --
    
    `pt.getLong("delay", 0L)`


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178788150
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -0,0 +1,111 @@
    +#!/usr/bin/env bash
    +
    +################################################################################
    +# 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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +
    +function checkLogs {
    +  parallelism=$1
    +  attempts=$2
    +  (( expectedCount=parallelism * (attempts + 1) ))
    +
    +  # Search for the log message that indicates restore problem from existing local state for the keyed backend.
    +  failedLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (2/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  # Search for attempts to recover locally.
    +  attemptLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (1/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  if [ ${failedLocalRecovery} -ne 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found ${failedLocalRecovery} failed attempt(s) for local recovery of correctly scheduled task(s)."
    +  fi
    +
    +  if [ ${attemptLocalRecovery} -eq 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found no attempt for local recovery. Configuration problem?"
    +  fi
    +}
    +
    +function cleanupAfterTest {
    +  # Reset the configurations
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=INFO, file/' "$FLINK_DIR/conf/log4j.properties"
    +  #
    +  kill ${watchdogPid} 2> /dev/null
    +  wait ${watchdogPid} 2> /dev/null
    +  #
    +  cleanup
    +}
    +
    +function cleanupAfterTestAndExitFail {
    +  cleanupAfterTest
    +  exit 1
    +}
    +
    +## This function executes one run for a certain configuration
    +function runLocalRecoveryTest {
    +  parallelism=$1
    +  maxAttempts=$2
    +  backend=$3
    +  incremental=$4
    +  killJVM=$5
    +
    +  echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${killJVM}."
    +  TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/target/flink-end-to-end-tests.jar
    +
    +  # Enable debug logging
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=DEBUG, file/' "$FLINK_DIR/conf/log4j.properties"
    +
    +  # Enable local recovery
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  echo "state.backend.local-recovery: ENABLE_FILE_BASED" >> "$FLINK_DIR/conf/flink-conf.yaml"
    +
    +  rm $FLINK_DIR/log/* 2> /dev/null
    +
    +  start_cluster
    +
    +  tm_watchdog ${parallelism} &
    +  watchdogPid=$!
    +
    +  echo "Started TM watchdog with PID ${watchdogPid}."
    +
    +  $FLINK_DIR/bin/flink run -c org.apache.flink.streaming.tests.StickyAllocationAndLocalRecoveryTestJob \
    +  -p ${parallelism} $TEST_PROGRAM_JAR --resolve-order parent-first \
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178771812
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175817764
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/NopTaskLocalStateStoreImpl.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.runtime.state;
    +
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.concurrent.CompletableFuture;
    +import java.util.function.LongPredicate;
    +
    +/**
    + * This class implements a {@link TaskLocalStateStore} with no functionality and is used when local recovery is
    + * disabled.
    + */
    +public final class NopTaskLocalStateStoreImpl implements OwnedTaskLocalStateStore {
    --- End diff --
    
    is `Nop` a typo and should be `NoOpTaskLocalStateStoreImpl`?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r187842776
  
    --- Diff: flink-core/src/main/java/org/apache/flink/configuration/JobManagerOptions.java ---
    @@ -140,7 +140,8 @@
     
     	public static final ConfigOption<Long> SLOT_IDLE_TIMEOUT =
     		key("slot.idle.timeout")
    -			.defaultValue(10L * 1000L)
    +			// default matches heartbeat.timeout so that sticky allocation is not lost on timeouts for local recovery
    +			.defaultValue(HeartbeatManagerOptions.HEARTBEAT_TIMEOUT.defaultValue())
    --- End diff --
    
    reminder: if this results in a different default value, I would suggest to make a note of the change in the "Release Notes" field of the JIRA ticket.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174446219
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,450 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt(runtimeContext.getAttemptNumber());
    +
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r187843886
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -0,0 +1,115 @@
    +#!/usr/bin/env bash
    +
    +################################################################################
    +# 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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +
    +# This function checks the logs for entries that indicate problems with local recovery
    +function check_logs {
    +  local parallelism=$1
    +  local attempts=$2
    +  (( expected_count=parallelism * (attempts + 1) ))
    +
    +  # Search for the log message that indicates restore problem from existing local state for the keyed backend.
    +  local failed_local_recovery=$(grep '^.*Creating keyed state backend.* from alternative (2/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  # Search for attempts to recover locally.
    +  local attempt_local_recovery=$(grep '^.*Creating keyed state backend.* from alternative (1/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  if [ ${failed_local_recovery} -ne 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found ${failed_local_recovery} failed attempt(s) for local recovery of correctly scheduled task(s)."
    +  fi
    +
    +  if [ ${attempt_local_recovery} -eq 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found no attempt for local recovery. Configuration problem?"
    +  fi
    +}
    +
    +# This function does a cleanup after the test. The configuration is restored, the watchdog is terminated and temporary
    +# files and folders are deleted.
    +function cleanup_after_test {
    +  # Reset the configurations
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=INFO, file/' "$FLINK_DIR/conf/log4j.properties"
    +  #
    +  kill ${watchdog_pid} 2> /dev/null
    +  wait ${watchdog_pid} 2> /dev/null
    +  #
    +  cleanup
    +}
    +
    +# Calls the cleanup step for this tests and exits with an error.
    +function cleanup_after_test_and_exit_fail {
    +  cleanup_after_test
    +  exit 1
    +}
    +
    +## This function executes one run for a certain configuration
    +function run_local_recovery_test {
    +  local parallelism=$1
    +  local max_attempts=$2
    +  local backend=$3
    +  local incremental=$4
    +  local kill_jvm=$5
    +
    +  echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${kill_jvm}."
    +  TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    +
    +  # Enable debug logging
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=DEBUG, file/' "$FLINK_DIR/conf/log4j.properties"
    +
    +  # Enable local recovery
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  echo "state.backend.local-recovery: ENABLE_FILE_BASED" >> "$FLINK_DIR/conf/flink-conf.yaml"
    --- End diff --
    
    FYI: we now have a `change_conf` method in `common.sh` to modify Flink configuration in e2e tests


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178836145
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    What I find problematic about deleting the DFS files is that not all state is (yet) covered by local recovery and it is also a lot harder to debug the cause if there is an actual scheduling problem. With the current code, you can easily see which allocation was lost. REST API might be an option if it is somehow exposed there.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175827727
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    --- End diff --
    
    Is his source supposed to run with a different parallelism than `1`? If yes, then it should implement the `RichParallelSourceFunction`.


---

[GitHub] flink issue #5676: [FLINK-8910][tests] Automated end-to-end test for local r...

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

    https://github.com/apache/flink/pull/5676
  
    I updated the PR to address the review comments. 
    
    @aljoscha maybe you can take a look because Till is not available for some time. 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174338580
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java ---
    @@ -105,6 +105,8 @@ public T createAndRestore(@Nonnull List<? extends Collection<S>> restoreOptions)
     
     			++alternativeIdx;
     
    +			// IMPORTANT: please be careful when modifying the log statements because they are used for validation in
    +			// the automatic end-to-end tests. Those tests might fail if they are not aligned with the log message!
     			if (restoreState.isEmpty()) {
     				LOG.debug("Creating {} with empty state.", logDescription);
     			} else {
    --- End diff --
    
    nit: since this log's format is so special, maybe we can introduce a `static final Sting LOG_FORMAT = "Creating {} with empty state."` for it, and guard it by adding a unit test (feel free to ignore).


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178819991
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    Previously, I had a "hacky" version that extracted the allocation id from a to string method of the state store but I think stephan was more in favour of exposing this information through the context if needed. I cannot see how we can test if we have the same allocation id without having some way to access it.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r176767603
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    Then we have a problem with other code parts as well, if I'm not mistaken.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178824035
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    Hmm, what about testing it a bit more indirectly, by removing the checkpoint files on the DFS. Then you can only recover if you recover locally. Or by querying the REST interface? We might have to add the information to the `VertexTaskDetail`.
    
    Otherwise, we start mixing concerns and expose unnecessary information to the user via the `AbstractRuntimeUDFContext`. Moreover, not every function has access to this information right now. For example the `RichAsyncFunctionRuntimeContext` does not expose it.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175832543
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    I'm also not sure whether we want to expose location information to the `Task`. Given scheduling transparency, this feels a bit wrong. Isn't there a different way to check whether a task was recovered locally?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175812511
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    +  expectedTm=$1
    +  while true;
    +  do
    +    runningTm=`jps | grep -o 'TaskManagerRunner' | wc -l`;
    +    count=$((expectedTm-runningTm))
    +    for (( c=0; c<count; c++ ))
    +    do
    +      $FLINK_DIR/bin/taskmanager.sh start > /dev/null
    +    done
    +    sleep 5;
    +  done
    +}
    +
    +function jm_kill_all {
    +  kill_all StandaloneSessionClusterEntrypoint
    +}
    +
    +function tm_kill_all {
    +  kill_all TaskManagerRunner
    --- End diff --
    
    Same here


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175825436
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    --- End diff --
    
    We can leave it here but in general I think abbreviations should be avoided.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

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


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175831034
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    Please attribute this code snippet if it comes from https://stackoverflow.com/a/12066696/4815083


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r182048477
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    @sihuazhou that approach would not allow us to test that the scheduling works proper when having more than one slot per TM, which is a useful thing to test for. Furthermore, this also silently makes the assumption that there is a connection between allocation id and PID, but the test might want to also validate that this is actually true.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175825728
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    --- End diff --
    
    wouldn't `pt.getBoolean("externalizedCheckpoints", false)` be the same?


---

[GitHub] flink issue #5676: [FLINK-8910][tests] Automated end-to-end test for local r...

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

    https://github.com/apache/flink/pull/5676
  
    CC @aljoscha 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175810393
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -31,12 +31,35 @@
     
     	private final String taskName;
     	private final String taskNameWithSubtasks;
    +	private final String allocationID;
     	private final int maxNumberOfParallelSubtasks;
     	private final int indexOfSubtask;
     	private final int numberOfParallelSubtasks;
     	private final int attemptNumber;
     
    -	public TaskInfo(String taskName, int maxNumberOfParallelSubtasks, int indexOfSubtask, int numberOfParallelSubtasks, int attemptNumber) {
    +	public TaskInfo(
    +		String taskName,
    +		int maxNumberOfParallelSubtasks,
    +		int indexOfSubtask,
    +		int numberOfParallelSubtasks,
    +		int attemptNumber) {
    --- End diff --
    
    Indentation is off by one level.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r182126796
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178790003
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r187845478
  
    --- Diff: flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,480 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + *
    + * <p>List of possible input parameters for this job:
    + * <ul>
    + * 	<li>checkpointDir: the checkpoint directory, required.</li>
    + * 	<li>parallelism: the parallelism of the job, default 1.</li>
    + *	<li>maxParallelism: the maximum parallelism of the job, default 1.</li>
    + * 	<li>checkpointInterval: the checkpointing interval in milliseconds, default 1000.</li>
    + * 	<li>restartDelay: the delay of the fixed delay restart strategy, default 0.</li>
    + * 	<li>externalizedCheckpoints: flag to activate externalized checkpoints, default <code>false</code>.</li>
    + * 	<li>stateBackend: choice for state backend between <code>file</code> and <code>rocks</code>, default <code>file</code>.</li>
    + * 	<li>killJvmOnFail: flag that determines whether or not an artificial failure induced by the test kills the JVM or not.</li>
    + * 	<li>asyncCheckpoints: flag for async checkpoints with file state backend, default <code>true</code>.</li>
    + * 	<li>incrementalCheckpoints: flag for incremental checkpoint with rocks state backend, default <code>false</code>.</li>
    + * 	<li>delay: sleep delay to throttle down the production of the source, default 0.</li>
    + * 	<li>maxAttempts: the maximum number of run attempts, before the job finishes with success, default 3.</li>
    + * 	<li>valueSize: size of the artificial value for each key in bytes, default 10.</li>
    + * </ul>
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", true);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.getLong("delay", 0L);
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.getInt("maxAttempts", 3);
    +
    +		// size of one artificial value
    +		int valueSize = pt.getInt("valueSize", 10);
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    --- End diff --
    
    Why is this map necessary for the test?


---

[GitHub] flink issue #5676: [FLINK-8910][tests] Automated end-to-end test for local r...

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

    https://github.com/apache/flink/pull/5676
  
    Thanks for the feedback, I have addressed the last comments and will merge this.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175828711
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    --- End diff --
    
    This field is not serializable and should thus be marked `transient`.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175812117
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    --- End diff --
    
    Please add a comment explaining what this function does. Especially in bash scripts, I think they can be very helpful.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175812319
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    +  expectedTm=$1
    +  while true;
    +  do
    +    runningTm=`jps | grep -o 'TaskManagerRunner' | wc -l`;
    --- End diff --
    
    Let's make the grep statement configurable because in non-Flip-6 mode the system does not start a `TaskManagerRunner` but a `TaskManager`.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r177695881
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    Thanks for clarifying the issue @greghogan. 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175826508
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    --- End diff --
    
    Same here.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r182116588
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175828150
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    --- End diff --
    
    element emission should always happen under the checkpoint lock


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r181540905
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    How about this way: setup the cluster with one slot per TM, and use the `host:pid` to act as the "`allocationID`"?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174446646
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java ---
    @@ -105,6 +105,8 @@ public T createAndRestore(@Nonnull List<? extends Collection<S>> restoreOptions)
     
     			++alternativeIdx;
     
    +			// IMPORTANT: please be careful when modifying the log statements because they are used for validation in
    +			// the automatic end-to-end tests. Those tests might fail if they are not aligned with the log message!
     			if (restoreState.isEmpty()) {
     				LOG.debug("Creating {} with empty state.", logDescription);
     			} else {
    --- End diff --
    
    I feel that guarding a log statement format by a test, just so that another test won't fail is a bit too much. I mean, it does not really break anything so a comment should be ok.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r176753796
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -0,0 +1,111 @@
    +#!/usr/bin/env bash
    +
    +################################################################################
    +# 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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +
    +function checkLogs {
    +  parallelism=$1
    +  attempts=$2
    +  (( expectedCount=parallelism * (attempts + 1) ))
    +
    +  # Search for the log message that indicates restore problem from existing local state for the keyed backend.
    +  failedLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (2/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  # Search for attempts to recover locally.
    +  attemptLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (1/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  if [ ${failedLocalRecovery} -ne 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found ${failedLocalRecovery} failed attempt(s) for local recovery of correctly scheduled task(s)."
    +  fi
    +
    +  if [ ${attemptLocalRecovery} -eq 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found no attempt for local recovery. Configuration problem?"
    +  fi
    +}
    +
    +function cleanupAfterTest {
    +  # Reset the configurations
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=INFO, file/' "$FLINK_DIR/conf/log4j.properties"
    +  #
    +  kill ${watchdogPid} 2> /dev/null
    +  wait ${watchdogPid} 2> /dev/null
    +  #
    --- End diff --
    
    The value `watchdogPid ` is not initialized here.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178770202
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    +  expectedTm=$1
    +  while true;
    +  do
    +    runningTm=`jps | grep -o 'TaskManagerRunner' | wc -l`;
    --- End diff --
    
    Do you think it would be ok to just grep for a regex that covers both names?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174446255
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,450 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt(runtimeContext.getAttemptNumber());
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt(int attempt) {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    +
    +		return (int) (Integer) pidMethod.invoke(mgmt);
    +	}
    +
    +	/**
    +	 * Records the information required to check sticky scheduling after a restart.
    +	 */
    +	private static class MapperSchedulingAndFailureInfo implements Serializable {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * True iff this task inflicts a test failure.
    +		 */
    +		final boolean failingTask;
    +
    +		/**
    +		 * True iff this task kills its JVM.
    +		 */
    +		final boolean killedJvm;
    +
    +		/**
    +		 * PID of the task JVM.
    +		 */
    +		final int jvmPid;
    +
    +		/**
    +		 * Name and stubtask index of the task.
    +		 */
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178770011
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    The question is not only whether it is local but also if it has the same allocation id, so I don't think there is really a different way of figuring this out.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178771748
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178789878
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175811121
  
    --- Diff: flink-core/src/main/java/org/apache/flink/api/common/TaskInfo.java ---
    @@ -107,4 +131,12 @@ public int getAttemptNumber() {
     	public String getTaskNameWithSubtasks() {
     		return this.taskNameWithSubtasks;
     	}
    +
    +	/**
    +	 * Returns the allocation id for where this task is executed.
    +	 * @return the allocation id for where this task is executed.
    +	 */
    +	public String getAllocationID() {
    --- End diff --
    
    If we really want to store the `AllocationID`, then we should store it not as a `String` but instead move `AllocationID` to `flink-core`. If this is not needed, then I would suggest to rename it something else.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175825995
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    --- End diff --
    
    Let's quickly document all the different configuration settings in the JavaDocs of the `main`. I think this makes it easier for the user of this class in the future.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174175230
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,450 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt(runtimeContext.getAttemptNumber());
    +
    --- End diff --
    
    nit: since we have used the `runtimeContext` to get some task infos (`numSubtasks`, `subtaskIdx`) in `shouldTaskFailForThisAttempt()`, maybe we can also use it to get the `attemptNum` in it.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178771774
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r187844104
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -0,0 +1,115 @@
    +#!/usr/bin/env bash
    +
    +################################################################################
    +# 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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +
    +# This function checks the logs for entries that indicate problems with local recovery
    +function check_logs {
    +  local parallelism=$1
    +  local attempts=$2
    +  (( expected_count=parallelism * (attempts + 1) ))
    +
    +  # Search for the log message that indicates restore problem from existing local state for the keyed backend.
    +  local failed_local_recovery=$(grep '^.*Creating keyed state backend.* from alternative (2/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  # Search for attempts to recover locally.
    +  local attempt_local_recovery=$(grep '^.*Creating keyed state backend.* from alternative (1/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  if [ ${failed_local_recovery} -ne 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found ${failed_local_recovery} failed attempt(s) for local recovery of correctly scheduled task(s)."
    +  fi
    +
    +  if [ ${attempt_local_recovery} -eq 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found no attempt for local recovery. Configuration problem?"
    +  fi
    +}
    +
    +# This function does a cleanup after the test. The configuration is restored, the watchdog is terminated and temporary
    +# files and folders are deleted.
    +function cleanup_after_test {
    +  # Reset the configurations
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=INFO, file/' "$FLINK_DIR/conf/log4j.properties"
    +  #
    +  kill ${watchdog_pid} 2> /dev/null
    +  wait ${watchdog_pid} 2> /dev/null
    +  #
    +  cleanup
    +}
    +
    +# Calls the cleanup step for this tests and exits with an error.
    +function cleanup_after_test_and_exit_fail {
    +  cleanup_after_test
    +  exit 1
    +}
    +
    +## This function executes one run for a certain configuration
    +function run_local_recovery_test {
    +  local parallelism=$1
    +  local max_attempts=$2
    +  local backend=$3
    +  local incremental=$4
    +  local kill_jvm=$5
    +
    +  echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${kill_jvm}."
    +  TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    +
    +  # Enable debug logging
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=DEBUG, file/' "$FLINK_DIR/conf/log4j.properties"
    +
    +  # Enable local recovery
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  echo "state.backend.local-recovery: ENABLE_FILE_BASED" >> "$FLINK_DIR/conf/flink-conf.yaml"
    +
    +  rm $FLINK_DIR/log/* 2> /dev/null
    +
    +  start_cluster
    +
    +  tm_watchdog ${parallelism} &
    +  watchdog_pid=$!
    +
    +  echo "Started TM watchdog with PID ${watchdog_pid}."
    +
    +  $FLINK_DIR/bin/flink run -c org.apache.flink.streaming.tests.StickyAllocationAndLocalRecoveryTestJob \
    +  -p ${parallelism} $TEST_PROGRAM_JAR \
    +  -D state.backend.local-recovery=ENABLE_FILE_BASED \
    +  --checkpointDir file://$TEST_DATA_DIR/local_recovery_test/checkpoints \
    +  --output $TEST_DATA_DIR/out/local_recovery_test/out --killJvmOnFail ${kill_jvm} --checkpointInterval 1000 \
    +  --maxAttempts ${max_attempts} --parallelism ${parallelism} --stateBackend ${backend} \
    +  --incrementalCheckpoints ${incremental}
    +
    +  check_logs ${parallelism} ${max_attempts}
    +  cleanup_after_test
    +}
    +
    +## MAIN
    +trap cleanup_after_test_and_exit_fail EXIT
    +#run_local_recovery_test 4 3 "file" "false" "false"
    +#run_local_recovery_test 4 3 "file" "false" "true"
    +#run_local_recovery_test 4 10 "rocks" "false" "false"
    +#run_local_recovery_test 4 10 "rocks" "true" "false"
    --- End diff --
    
    Do these need to be re-enabled / removed?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178789236
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    --- End diff --
    
    True, some of these parameter extractions were just copied from the state machine job and could be simplified.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175812472
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    +  expectedTm=$1
    +  while true;
    +  do
    +    runningTm=`jps | grep -o 'TaskManagerRunner' | wc -l`;
    +    count=$((expectedTm-runningTm))
    +    for (( c=0; c<count; c++ ))
    +    do
    +      $FLINK_DIR/bin/taskmanager.sh start > /dev/null
    +    done
    +    sleep 5;
    +  done
    +}
    +
    +function jm_kill_all {
    +  kill_all StandaloneSessionClusterEntrypoint
    --- End diff --
    
    Would be good to make it work with the legacy code as well.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175817052
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -0,0 +1,111 @@
    +#!/usr/bin/env bash
    +
    +################################################################################
    +# 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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +
    +function checkLogs {
    +  parallelism=$1
    +  attempts=$2
    +  (( expectedCount=parallelism * (attempts + 1) ))
    +
    +  # Search for the log message that indicates restore problem from existing local state for the keyed backend.
    +  failedLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (2/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  # Search for attempts to recover locally.
    +  attemptLocalRecovery=$(grep '^.*Creating keyed state backend.* from alternative (1/2)\.$' $FLINK_DIR/log/* | wc -l | tr -d ' ')
    +
    +  if [ ${failedLocalRecovery} -ne 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found ${failedLocalRecovery} failed attempt(s) for local recovery of correctly scheduled task(s)."
    +  fi
    +
    +  if [ ${attemptLocalRecovery} -eq 0 ]
    +  then
    +    PASS=""
    +    echo "FAILURE: Found no attempt for local recovery. Configuration problem?"
    +  fi
    +}
    +
    +function cleanupAfterTest {
    +  # Reset the configurations
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=INFO, file/' "$FLINK_DIR/conf/log4j.properties"
    +  #
    +  kill ${watchdogPid} 2> /dev/null
    +  wait ${watchdogPid} 2> /dev/null
    +  #
    +  cleanup
    +}
    +
    +function cleanupAfterTestAndExitFail {
    +  cleanupAfterTest
    +  exit 1
    +}
    +
    +## This function executes one run for a certain configuration
    +function runLocalRecoveryTest {
    +  parallelism=$1
    +  maxAttempts=$2
    +  backend=$3
    +  incremental=$4
    +  killJVM=$5
    +
    +  echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${killJVM}."
    +  TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/target/flink-end-to-end-tests.jar
    +
    +  # Enable debug logging
    +  sed -i -e 's/log4j.rootLogger=.*/log4j.rootLogger=DEBUG, file/' "$FLINK_DIR/conf/log4j.properties"
    +
    +  # Enable local recovery
    +  sed -i -e 's/state.backend.local-recovery: .*//' "$FLINK_DIR/conf/flink-conf.yaml"
    +  echo "state.backend.local-recovery: ENABLE_FILE_BASED" >> "$FLINK_DIR/conf/flink-conf.yaml"
    +
    +  rm $FLINK_DIR/log/* 2> /dev/null
    +
    +  start_cluster
    +
    +  tm_watchdog ${parallelism} &
    +  watchdogPid=$!
    +
    +  echo "Started TM watchdog with PID ${watchdogPid}."
    +
    +  $FLINK_DIR/bin/flink run -c org.apache.flink.streaming.tests.StickyAllocationAndLocalRecoveryTestJob \
    +  -p ${parallelism} $TEST_PROGRAM_JAR --resolve-order parent-first \
    --- End diff --
    
    why do we start the job with `--resolve-order parent-first`?


---

[GitHub] flink issue #5676: [FLINK-8910][tests] Automated end-to-end test for local r...

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

    https://github.com/apache/flink/pull/5676
  
    CC @tillrohrmann 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r174165460
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,450 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt(runtimeContext.getAttemptNumber());
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt(int attempt) {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    +
    +		return (int) (Integer) pidMethod.invoke(mgmt);
    +	}
    +
    +	/**
    +	 * Records the information required to check sticky scheduling after a restart.
    +	 */
    +	private static class MapperSchedulingAndFailureInfo implements Serializable {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * True iff this task inflicts a test failure.
    +		 */
    +		final boolean failingTask;
    +
    +		/**
    +		 * True iff this task kills its JVM.
    +		 */
    +		final boolean killedJvm;
    +
    +		/**
    +		 * PID of the task JVM.
    +		 */
    +		final int jvmPid;
    +
    +		/**
    +		 * Name and stubtask index of the task.
    +		 */
    --- End diff --
    
    typo: `stubtask` -> `subtask`.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178790739
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    +
    +		/**
    +		 * Generator's current key.
    +		 */
    +		long currentKey;
    +
    +		/**
    +		 * Generator runs while this is true.
    +		 */
    +		volatile boolean running;
    +
    +		RandomLongSource(int maxAttempts, long delay) {
    +			this.delay = delay;
    +			this.maxAttempts = maxAttempts;
    +			this.running = true;
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Long> sourceContext) throws Exception {
    +
    +			int numberOfParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks();
    +			int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask();
    +
    +			// the source emits one final event and shuts down once we have reached max attempts.
    +			if (getRuntimeContext().getAttemptNumber() > maxAttempts) {
    +				sourceContext.collect(Long.MAX_VALUE - subtaskIdx);
    +				return;
    +			}
    +
    +			while (running) {
    +				sourceContext.collect(currentKey);
    +				currentKey += numberOfParallelSubtasks;
    +
    +				if (delay > 0) {
    +					Thread.sleep(delay);
    +				}
    +			}
    +		}
    +
    +		@Override
    +		public void cancel() {
    +			running = false;
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +			sourceCurrentKeyState.clear();
    +			sourceCurrentKeyState.add(currentKey);
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext context) throws Exception {
    +
    +			ListStateDescriptor<Long> currentKeyDescriptor = new ListStateDescriptor<>("currentKey", Long.class);
    +			sourceCurrentKeyState = context.getOperatorStateStore().getListState(currentKeyDescriptor);
    +
    +			currentKey = getRuntimeContext().getIndexOfThisSubtask();
    +			Iterable<Long> iterable = sourceCurrentKeyState.get();
    +			if (iterable != null) {
    +				Iterator<Long> iterator = iterable.iterator();
    +				if (iterator.hasNext()) {
    +					currentKey = iterator.next();
    +					Preconditions.checkState(!iterator.hasNext());
    +				}
    +			}
    +		}
    +	}
    +
    +	/**
    +	 * Stateful map function. Failure creation and checks happen here.
    +	 */
    +	private static final class StateCreatingFlatMap
    +		extends RichFlatMapFunction<Long, String> implements CheckpointedFunction, CheckpointListener {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * User configured size of the generated artificial values in the keyed state.
    +		 */
    +		final int valueSize;
    +
    +		/**
    +		 * Holds the user configuration if the artificial test failure is killing the JVM.
    +		 */
    +		final boolean killTaskOnFailure;
    +
    +		/**
    +		 * This state is used to create artificial keyed state in the backend.
    +		 */
    +		transient ValueState<String> valueState;
    +
    +		/**
    +		 * This state is used to persist the schedulingAndFailureInfo to state.
    +		 */
    +		transient ListState<MapperSchedulingAndFailureInfo> schedulingAndFailureState;
    +
    +		/**
    +		 * This contains the current scheduling and failure meta data.
    +		 */
    +		transient MapperSchedulingAndFailureInfo currentSchedulingAndFailureInfo;
    +
    +		/**
    +		 * Message to indicate that recovery detected a failure with sticky allocation.
    +		 */
    +		transient volatile String allocationFailureMessage;
    +
    +		/**
    +		 * If this flag is true, the next invocation of the map function introduces a test failure.
    +		 */
    +		transient volatile boolean failTask;
    +
    +		StateCreatingFlatMap(int valueSize, boolean killTaskOnFailure) {
    +			this.valueSize = valueSize;
    +			this.failTask = false;
    +			this.killTaskOnFailure = killTaskOnFailure;
    +			this.allocationFailureMessage = null;
    +		}
    +
    +		@Override
    +		public void flatMap(Long key, Collector<String> collector) throws IOException {
    +
    +			if (allocationFailureMessage != null) {
    +				// Report the failure downstream, so that we can get the message from the output.
    +				collector.collect(allocationFailureMessage);
    +				allocationFailureMessage = null;
    +			}
    +
    +			if (failTask) {
    +				// we fail the task, either by killing the JVM hard, or by throwing a user code exception.
    +				if (killTaskOnFailure) {
    +					Runtime.getRuntime().halt(-1);
    +				} else {
    +					throw new RuntimeException("Artificial user code exception.");
    +				}
    +			}
    +
    +			// sanity check
    +			if (null != valueState.value()) {
    +				throw new IllegalStateException("This should never happen, keys are generated monotonously.");
    +			}
    +
    +			// store artificial data to blow up the state
    +			valueState.update(RandomStringUtils.random(valueSize, true, true));
    +		}
    +
    +		@Override
    +		public void snapshotState(FunctionSnapshotContext functionSnapshotContext) {
    +		}
    +
    +		@Override
    +		public void initializeState(FunctionInitializationContext functionInitializationContext) throws Exception {
    +
    +			ValueStateDescriptor<String> stateDescriptor =
    +				new ValueStateDescriptor<>("state", String.class);
    +			valueState = functionInitializationContext.getKeyedStateStore().getState(stateDescriptor);
    +
    +			ListStateDescriptor<MapperSchedulingAndFailureInfo> mapperInfoStateDescriptor =
    +				new ListStateDescriptor<>("mapperState", MapperSchedulingAndFailureInfo.class);
    +			schedulingAndFailureState =
    +				functionInitializationContext.getOperatorStateStore().getUnionListState(mapperInfoStateDescriptor);
    +
    +			StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext();
    +			String allocationID = runtimeContext.getAllocationID();
    +
    +			final int thisJvmPid = getJvmPid();
    +			final Set<Integer> killedJvmPids = new HashSet<>();
    +
    +			// here we check if the sticky scheduling worked as expected
    +			if (functionInitializationContext.isRestored()) {
    +				Iterable<MapperSchedulingAndFailureInfo> iterable = schedulingAndFailureState.get();
    +				String taskNameWithSubtasks = runtimeContext.getTaskNameWithSubtasks();
    +
    +				MapperSchedulingAndFailureInfo infoForThisTask = null;
    +				List<MapperSchedulingAndFailureInfo> completeInfo = new ArrayList<>();
    +				if (iterable != null) {
    +					for (MapperSchedulingAndFailureInfo testInfo : iterable) {
    +
    +						completeInfo.add(testInfo);
    +
    +						if (taskNameWithSubtasks.equals(testInfo.taskNameWithSubtask)) {
    +							infoForThisTask = testInfo;
    +						}
    +
    +						if (testInfo.killedJvm) {
    +							killedJvmPids.add(testInfo.jvmPid);
    +						}
    +					}
    +				}
    +
    +				Preconditions.checkNotNull(infoForThisTask, "Expected to find info here.");
    +
    +				if (!isScheduledToCorrectAllocation(infoForThisTask, allocationID, killedJvmPids)) {
    +					allocationFailureMessage = String.format(
    +						"Sticky allocation test failed: Subtask %s in attempt %d was rescheduled from allocation %s " +
    +							"on JVM with PID %d to unexpected allocation %s on JVM with PID %d.\n" +
    +							"Complete information from before the crash: %s.",
    +						runtimeContext.getTaskNameWithSubtasks(),
    +						runtimeContext.getAttemptNumber(),
    +						infoForThisTask.allocationId,
    +						infoForThisTask.jvmPid,
    +						allocationID,
    +						thisJvmPid,
    +						completeInfo);
    +				}
    +			}
    +
    +			// We determine which of the subtasks will produce the artificial failure
    +			boolean failingTask = shouldTaskFailForThisAttempt();
    +
    +			// We take note of all the meta info that we require to check sticky scheduling in the next re-attempt
    +			this.currentSchedulingAndFailureInfo = new MapperSchedulingAndFailureInfo(
    +				failingTask,
    +				failingTask && killTaskOnFailure,
    +				thisJvmPid,
    +				runtimeContext.getTaskNameWithSubtasks(),
    +				allocationID);
    +
    +			schedulingAndFailureState.clear();
    +			schedulingAndFailureState.add(currentSchedulingAndFailureInfo);
    +		}
    +
    +		@Override
    +		public void notifyCheckpointComplete(long checkpointId) {
    +			// we can only fail the task after at least one checkpoint is completed to record progress.
    +			failTask = currentSchedulingAndFailureInfo.failingTask;
    +		}
    +
    +		private boolean shouldTaskFailForThisAttempt() {
    +			RuntimeContext runtimeContext = getRuntimeContext();
    +			int numSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +			int subtaskIdx = runtimeContext.getIndexOfThisSubtask();
    +			int attempt = runtimeContext.getAttemptNumber();
    +			return (attempt % numSubtasks) == subtaskIdx;
    +		}
    +
    +		private boolean isScheduledToCorrectAllocation(
    +			MapperSchedulingAndFailureInfo infoForThisTask,
    +			String allocationID,
    +			Set<Integer> killedJvmPids) {
    +
    +			return (infoForThisTask.allocationId.equals(allocationID)
    +				|| killedJvmPids.contains(infoForThisTask.jvmPid));
    +		}
    +	}
    +
    +	private static int getJvmPid() throws Exception {
    +		java.lang.management.RuntimeMXBean runtime =
    +			java.lang.management.ManagementFactory.getRuntimeMXBean();
    +		java.lang.reflect.Field jvm = runtime.getClass().getDeclaredField("jvm");
    +		jvm.setAccessible(true);
    +		sun.management.VMManagement mgmt =
    +			(sun.management.VMManagement) jvm.get(runtime);
    +		java.lang.reflect.Method pidMethod =
    +			mgmt.getClass().getDeclaredMethod("getProcessId");
    +		pidMethod.setAccessible(true);
    --- End diff --
    
    👍 added attributions. In case of license problems, I wonder how this would work for such trivial code that is just calling three getters in an API?


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r182117045
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178790036
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r182116943
  
    --- Diff: flink-end-to-end-tests/test-scripts/common.sh ---
    @@ -176,10 +176,40 @@ function s3_delete {
         https://${bucket}.s3.amazonaws.com/${s3_file}
     }
     
    +function tm_watchdog {
    +  expectedTm=$1
    +  while true;
    +  do
    +    runningTm=`jps | grep -o 'TaskManagerRunner' | wc -l`;
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178788782
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/NopTaskLocalStateStoreImpl.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.runtime.state;
    +
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.concurrent.CompletableFuture;
    +import java.util.function.LongPredicate;
    +
    +/**
    + * This class implements a {@link TaskLocalStateStore} with no functionality and is used when local recovery is
    + * disabled.
    + */
    +public final class NopTaskLocalStateStoreImpl implements OwnedTaskLocalStateStore {
    --- End diff --
    
    I thought `Nop` is a well-established shortcut for `NoOp`, but also don't mind to rename.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r175826610
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    --- End diff --
    
    Same here.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178790018
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178804348
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/NopTaskLocalStateStoreImpl.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.runtime.state;
    +
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.concurrent.CompletableFuture;
    +import java.util.function.LongPredicate;
    +
    +/**
    + * This class implements a {@link TaskLocalStateStore} with no functionality and is used when local recovery is
    + * disabled.
    + */
    +public final class NopTaskLocalStateStoreImpl implements OwnedTaskLocalStateStore {
    --- End diff --
    
    My bad.


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178771947
  
    --- Diff: flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java ---
    @@ -0,0 +1,451 @@
    +/**
    + * 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
    + * <p>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p>
    + * 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.tests;
    +
    +import org.apache.flink.api.common.functions.MapFunction;
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.common.restartstrategy.RestartStrategies;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.state.ValueState;
    +import org.apache.flink.api.common.state.ValueStateDescriptor;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.CheckpointConfig;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.Preconditions;
    +
    +import org.apache.commons.lang3.RandomStringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +/**
    + * Automatic end-to-end test for local recovery (including sticky allocation).
    + */
    +public class StickyAllocationAndLocalRecoveryTestJob {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(StickyAllocationAndLocalRecoveryTestJob.class);
    +
    +	public static void main(String[] args) throws Exception {
    +
    +		final ParameterTool pt = ParameterTool.fromArgs(args);
    +
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		env.setParallelism(pt.getInt("parallelism", 1));
    +		env.setMaxParallelism(pt.getInt("maxParallelism", pt.getInt("parallelism", 1)));
    +		env.enableCheckpointing(pt.getInt("checkpointInterval", 1000));
    +		env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, pt.getInt("restartDelay", 0)));
    +		if (pt.has("externalizedCheckpoints") && pt.getBoolean("externalizedCheckpoints", false)) {
    +			env.getCheckpointConfig().enableExternalizedCheckpoints(CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
    +		}
    +
    +		String stateBackend = pt.get("stateBackend", "file");
    +		String checkpointDir = pt.getRequired("checkpointDir");
    +
    +		boolean killJvmOnFail = pt.getBoolean("killJvmOnFail", false);
    +
    +		if ("file".equals(stateBackend)) {
    +			boolean asyncCheckpoints = pt.getBoolean("asyncCheckpoints", false);
    +			env.setStateBackend(new FsStateBackend(checkpointDir, asyncCheckpoints));
    +		} else if ("rocks".equals(stateBackend)) {
    +			boolean incrementalCheckpoints = pt.getBoolean("incrementalCheckpoints", false);
    +			env.setStateBackend(new RocksDBStateBackend(checkpointDir, incrementalCheckpoints));
    +		} else {
    +			throw new IllegalArgumentException("Unknown backend: " + stateBackend);
    +		}
    +
    +		// make parameters available in the web interface
    +		env.getConfig().setGlobalJobParameters(pt);
    +
    +		// delay to throttle down the production of the source
    +		long delay = pt.has("delay") ? pt.getLong("delay") : 0L;
    +
    +		// the maximum number of attempts, before the job finishes with success
    +		int maxAttempts = pt.has("maxAttempts") ? pt.getInt("maxAttempts") : 3;
    +
    +		// size of one artificial value
    +		int valueSize = pt.has("valueSize") ? pt.getInt("valueSize") : 10;
    +
    +		env.addSource(new RandomLongSource(maxAttempts, delay))
    +			.keyBy((KeySelector<Long, Long>) aLong -> aLong)
    +			.flatMap(new StateCreatingFlatMap(valueSize, killJvmOnFail))
    +			.map((MapFunction<String, String>) value -> value)
    +			.addSink(new PrintSinkFunction<>());
    +
    +		env.execute("Sticky Allocation And Local Recovery Test");
    +	}
    +
    +	/**
    +	 * Source function that produces a long sequence.
    +	 */
    +	private static final class RandomLongSource extends RichSourceFunction<Long> implements CheckpointedFunction {
    +
    +		private static final long serialVersionUID = 1L;
    +
    +		/**
    +		 * Generator delay between two events.
    +		 */
    +		final long delay;
    +
    +		/**
    +		 * Maximum restarts before shutting down this source.
    +		 */
    +		final int maxAttempts;
    +
    +		/**
    +		 * State that holds the current key for recovery.
    +		 */
    +		ListState<Long> sourceCurrentKeyState;
    --- End diff --
    
    👍 


---

[GitHub] flink pull request #5676: [FLINK-8910][tests] Automated end-to-end test for ...

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

    https://github.com/apache/flink/pull/5676#discussion_r178789937
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/NopTaskLocalStateStoreImpl.java ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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.runtime.state;
    +
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.util.concurrent.CompletableFuture;
    +import java.util.function.LongPredicate;
    +
    +/**
    + * This class implements a {@link TaskLocalStateStore} with no functionality and is used when local recovery is
    + * disabled.
    + */
    +public final class NopTaskLocalStateStoreImpl implements OwnedTaskLocalStateStore {
    --- End diff --
    
    👍 


---