Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/2252#discussion_r70956813 --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java --- @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.tasks; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.filecache.FileCache; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.SerializableObject; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.util.SerializedValue; + +import org.junit.Test; + +import scala.concurrent.duration.FiniteDuration; + +import java.io.IOException; +import java.io.Serializable; +import java.net.URL; +import java.util.Collections; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +/** + * This test checks that task restores that get stuck in the presence of interrupts + * are handled properly. + * + * In practice, reading from HDFS is interrupt sensitive: The HDFS code frequently deadlocks + * or livelocks if it is interrupted. + */ +public class InterruptSensitiveRestoreTest { + + private static final OneShotLatch IN_RESTORE_LATCH = new OneShotLatch(); + + @Test + public void testRestoreWithInterrupt() throws Exception { + + Configuration taskConfig = new Configuration(); + StreamConfig cfg = new StreamConfig(taskConfig); + cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + cfg.setStreamOperator(new StreamSource<>(new TestSource())); + + StateHandle<Serializable> lockingHandle = new InterruptLockingStateHandle(); + StreamTaskState opState = new StreamTaskState(); + opState.setFunctionState(lockingHandle); + StreamTaskStateList taskState = new StreamTaskStateList(new StreamTaskState[] { opState }); + + TaskDeploymentDescriptor tdd = createTaskDeploymentDescriptor(taskConfig, taskState); + Task task = createTask(tdd); + + // start the task and wait until it is in "restore" + task.startTaskThread(); + IN_RESTORE_LATCH.await(); + + // trigger cancellation and signal to continue + task.cancelExecution(); + + task.getExecutingThread().join(30000); --- End diff -- I think it should not be a race. On interruption, the thread's 'interrupt' flag will be set and, upon entering `wait()`, it should immediately throw an `InterruptedException`. Also, the cancellation sends periodic interrupts.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---