From 3111a648c4bd0846938c11a7cc93362b009ac1d7 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Thu, 3 Sep 2015 15:13:28 +0200 Subject: [PATCH 1/7] [runtime] Add type parameter to ByteStreamStateHandle --- .../runtime/state/ByteStreamStateHandle.java | 23 ++++++++----------- .../flink/runtime/state/FileStateHandle.java | 19 ++++++++------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java index 7ecfe62440786..23cc54ae065dc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java @@ -32,14 +32,14 @@ * serialization. * */ -public abstract class ByteStreamStateHandle implements StateHandle { +public abstract class ByteStreamStateHandle implements StateHandle { private static final long serialVersionUID = -962025800339325828L; - private transient Serializable state; + private transient T state; private boolean isWritten = false; - public ByteStreamStateHandle(Serializable state) { + public ByteStreamStateHandle(T state) { if (state != null) { this.state = state; } else { @@ -58,13 +58,13 @@ public ByteStreamStateHandle(Serializable state) { protected abstract InputStream getInputStream() throws Exception; @Override - public Serializable getState(ClassLoader userCodeClassLoader) throws Exception { + @SuppressWarnings("unchecked") + public T getState(ClassLoader userCodeClassLoader) throws Exception { if (!stateFetched()) { - ObjectInputStream stream = new InstantiationUtil.ClassLoaderObjectInputStream(getInputStream(), userCodeClassLoader); - try { - state = (Serializable) stream.readObject(); - } finally { - stream.close(); + try (ObjectInputStream stream = new InstantiationUtil + .ClassLoaderObjectInputStream(getInputStream(), userCodeClassLoader)) { + + state = (T) stream.readObject(); } } return state; @@ -72,12 +72,9 @@ public Serializable getState(ClassLoader userCodeClassLoader) throws Exception { private void writeObject(ObjectOutputStream oos) throws Exception { if (!isWritten) { - ObjectOutputStream stream = new ObjectOutputStream(getOutputStream()); - try { + try (ObjectOutputStream stream = new ObjectOutputStream(getOutputStream())) { stream.writeObject(state); isWritten = true; - } finally { - stream.close(); } } oos.defaultWriteObject(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java index 091c739228bdc..8d22273822e41 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java @@ -31,6 +31,8 @@ import java.util.Random; +import static com.google.common.base.Preconditions.checkNotNull; + /** * Statehandle that writes the checkpointed state to a random file in the * provided checkpoint directory. Any Flink supported File system can be used @@ -38,13 +40,13 @@ * failures, such as HDFS or Tachyon. * */ -public class FileStateHandle extends ByteStreamStateHandle { +public class FileStateHandle extends ByteStreamStateHandle { private static final long serialVersionUID = 1L; private String pathString; - public FileStateHandle(Serializable state, String folder) { + public FileStateHandle(T state, String folder) { super(state); this.pathString = folder + "/" + randomString(); } @@ -73,8 +75,8 @@ public void discardState() throws Exception { * {@link FileStateHandle}s for a given checkpoint directory. * */ - public static StateHandleProvider createProvider(String checkpointDir) { - return new FileStateHandleProvider(checkpointDir); + public static StateHandleProvider createProvider(String checkpointDir) { + return new FileStateHandleProvider<>(checkpointDir); } /** @@ -82,18 +84,19 @@ public static StateHandleProvider createProvider(String checkpoint * given checkpoint directory. * */ - private static class FileStateHandleProvider implements StateHandleProvider { + private static class FileStateHandleProvider implements StateHandleProvider { private static final long serialVersionUID = 3496670017955260518L; private String path; public FileStateHandleProvider(String path) { - this.path = path; + this.path = checkNotNull(path, "Path"); } @Override - public FileStateHandle createStateHandle(Serializable state) { - return new FileStateHandle(state, path); + @SuppressWarnings("unchecked") + public FileStateHandle createStateHandle(T state) { + return (FileStateHandle) new FileStateHandle(state, path); } } From c41999e5177c26e99fc9ffc7035e067fd3f792d5 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Sun, 20 Sep 2015 13:08:24 +0200 Subject: [PATCH 2/7] [FLINK-2652] [tests] Temporary ignore flakey PartitionRequestClientFactoryTest --- .../io/network/netty/PartitionRequestClientFactoryTest.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java index 24a2a5cd7e8d2..56e5bdebb91f8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java @@ -25,6 +25,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.ConnectionID; +import org.apache.flink.runtime.net.NetUtils; +import org.junit.Ignore; import org.junit.Test; import java.io.IOException; @@ -41,9 +43,10 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +@Ignore public class PartitionRequestClientFactoryTest { - private final static int SERVER_PORT = 10021; + private final static int SERVER_PORT = NetUtils.getAvailablePort(); @Test public void testResourceReleaseAfterInterruptedConnect() throws Exception { From 02ab42db87418fd7c349d6a26b630b69c91aa40b Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 30 Sep 2015 16:38:37 +0200 Subject: [PATCH 3/7] [FLINK-2792] [jobmanager, logging] Set actor message log level to TRACE --- .../main/scala/org/apache/flink/runtime/LogMessages.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala index e1e8961ab645a..e4c17473463ed 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/LogMessages.scala @@ -29,18 +29,18 @@ trait LogMessages extends FlinkActor { override def isDefinedAt(x: Any): Boolean = _receive.isDefinedAt(x) override def apply(x: Any): Unit = { - if (!log.isDebugEnabled) { + if (!log.isTraceEnabled) { _receive(x) } else { - log.debug(s"Received message $x at ${context.self.path} from ${context.sender()}.") + log.trace(s"Received message $x at ${context.self.path} from ${context.sender()}.") val start = System.nanoTime() _receive(x) val duration = (System.nanoTime() - start) / 1000000 - log.debug(s"Handled message $x in $duration ms from ${context.sender()}.") + log.trace(s"Handled message $x in $duration ms from ${context.sender()}.") } } } From ce6a943b31b71e790c0370d0a405891453a9a344 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Tue, 1 Sep 2015 17:25:46 +0200 Subject: [PATCH 4/7] [FLINK-2354] [runtime] Add job graph and checkpoint recovery This closes #1153. --- .../flink/configuration/ConfigConstants.java | 21 + .../checkpoint/CheckpointCoordinator.java | 223 +++--- .../checkpoint/CheckpointIDCounter.java | 43 ++ .../checkpoint/CheckpointRecoveryFactory.java | 61 ++ ...eckpoint.java => CompletedCheckpoint.java} | 17 +- .../checkpoint/CompletedCheckpointStore.java | 69 ++ .../runtime/checkpoint/PendingCheckpoint.java | 6 +- .../StandaloneCheckpointIDCounter.java | 47 ++ .../StandaloneCheckpointRecoveryFactory.java | 52 ++ .../StandaloneCompletedCheckpointStore.java | 100 +++ .../runtime/checkpoint/StateForTask.java | 21 +- .../ZooKeeperCheckpointIDCounter.java | 130 ++++ .../ZooKeeperCheckpointRecoveryFactory.java | 66 ++ .../ZooKeeperCompletedCheckpointStore.java | 293 +++++++ .../executiongraph/ExecutionGraph.java | 51 +- .../flink/runtime/jobgraph/JobGraph.java | 5 + .../runtime/jobmanager/RecoveryMode.java | 13 + .../StandaloneSubmittedJobGraphStore.java | 65 ++ .../runtime/jobmanager/SubmittedJobGraph.java | 77 ++ .../jobmanager/SubmittedJobGraphStore.java | 93 +++ .../ZooKeeperSubmittedJobGraphStore.java | 379 ++++++++++ .../leaderelection/LeaderElectionService.java | 1 + .../ZooKeeperLeaderElectionService.java | 1 + .../flink/runtime/state/StateBackend.java | 39 + .../state/StateHandleProviderFactory.java | 61 ++ .../runtime/util/LeaderElectionUtils.java | 67 -- .../flink/runtime/util/ZooKeeperUtils.java | 138 +++- .../zookeeper/ZooKeeperStateHandleStore.java | 384 ++++++++++ .../flink/runtime/jobmanager/JobInfo.scala | 25 +- .../flink/runtime/jobmanager/JobManager.scala | 596 ++++++++++----- .../runtime/messages/JobManagerMessages.scala | 16 + .../minicluster/FlinkMiniCluster.scala | 4 +- .../checkpoint/CheckpointCoordinatorTest.java | 190 +++-- .../checkpoint/CheckpointIDCounterTest.java | 194 +++++ .../CheckpointStateRestoreTest.java | 19 +- .../CompletedCheckpointStoreTest.java | 297 ++++++++ ...tandaloneCompletedCheckpointStoreTest.java | 33 + ...oKeeperCompletedCheckpointStoreITCase.java | 101 +++ .../BlobLibraryCacheManagerTest.java | 4 + .../PartitionRequestClientFactoryTest.java | 2 +- ...nagerSubmittedJobGraphsRecoveryITCase.java | 460 +++++++++++ .../StandaloneSubmittedJobGraphStoreTest.java | 53 ++ ...ooKeeperSubmittedJobGraphsStoreITCase.java | 283 +++++++ .../JobManagerLeaderElectionTest.java | 27 +- .../TestingLeaderElectionService.java | 4 + .../ZooKeeperLeaderRetrievalTest.java | 21 +- .../messages/CheckpointMessagesTest.java | 2 +- .../runtime/taskmanager/TaskCancelTest.java | 67 +- .../runtime/testutils/CommonTestUtils.java | 75 +- .../testutils/JobManagerActorTestUtils.java | 166 ++++ .../runtime/testutils/JobManagerProcess.java | 226 ++++++ .../runtime/testutils/TaskManagerProcess.java | 133 ++++ .../runtime/testutils/TestJvmProcess.java | 267 +++++++ .../runtime/testutils/ZooKeeperTestUtils.java | 94 +++ .../ZooKeeperStateHandleStoreITCase.java | 591 +++++++++++++++ .../zookeeper/ZooKeeperTestEnvironment.java | 133 ++++ .../ExecutionGraphRestartTest.scala | 21 +- .../runtime/testingUtils/TestingCluster.scala | 44 +- .../testingUtils/TestingJobManager.scala | 32 +- .../streaming/runtime/tasks/StreamTask.java | 5 +- .../test/util/ForkableFlinkMiniCluster.scala | 36 +- ...Case.java => StateCheckpointedITCase.java} | 4 +- ...obManagerProcessFailureRecoveryITCase.java | 289 +++++++ ...askManagerProcessFailureRecoveryTest.java} | 69 +- .../test/recovery/ChaosMonkeyITCase.java | 713 ++++++++++++++++++ .../JobManagerCheckpointRecoveryITCase.java | 395 ++++++++++ ...agerProcessFailureBatchRecoveryITCase.java | 140 ++++ .../ProcessFailureCancelingITCase.java | 4 +- ...gerProcessFailureBatchRecoveryITCase.java} | 6 +- ...rocessFailureStreamingRecoveryITCase.java} | 18 +- .../ZooKeeperLeaderElectionITCase.java | 61 +- .../apache/flink/yarn/YarnJobManager.scala | 11 +- 72 files changed, 7717 insertions(+), 737 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java rename flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/{SuccessfulCheckpoint.java => CompletedCheckpoint.java} (85%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java rename flink-tests/src/test/java/org/apache/flink/test/checkpointing/{StateCheckpoinedITCase.java => StateCheckpointedITCase.java} (99%) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java rename flink-tests/src/test/java/org/apache/flink/test/recovery/{AbstractProcessFailureRecoveryTest.java => AbstractTaskManagerProcessFailureRecoveryTest.java} (87%) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java rename flink-tests/src/test/java/org/apache/flink/test/recovery/{ProcessFailureBatchRecoveryITCase.java => TaskManagerProcessFailureBatchRecoveryITCase.java} (92%) rename flink-tests/src/test/java/org/apache/flink/test/recovery/{ProcessFailureStreamingRecoveryITCase.java => TaskManagerProcessFailureStreamingRecoveryITCase.java} (94%) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 36369abd2d94e..288a4548f7f03 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -419,6 +419,12 @@ public final class ConfigConstants { * Directory for saving streaming checkpoints */ public static final String STATE_BACKEND_FS_DIR = "state.backend.fs.checkpointdir"; + + /** + * File system state backend base path for recoverable state handles. Recovery state is written + * to this path and the file state handles are persisted for recovery. + */ + public static final String STATE_BACKEND_FS_RECOVERY_PATH = "state.backend.fs.dir.recovery"; // ----------------------------- Miscellaneous ---------------------------- @@ -446,6 +452,15 @@ public final class ConfigConstants { public static final String ZOOKEEPER_LEADER_PATH = "ha.zookeeper.dir.leader"; + /** ZooKeeper root path (ZNode) for job graphs. */ + public static final String ZOOKEEPER_JOBGRAPHS_PATH = "ha.zookeeper.dir.jobgraphs"; + + /** ZooKeeper root path (ZNode) for completed checkpoints. */ + public static final String ZOOKEEPER_CHECKPOINTS_PATH = "ha.zookeeper.dir.checkpoints"; + + /** ZooKeeper root path (ZNode) for checkpoint counters. */ + public static final String ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "ha.zookeeper.dir.checkpoint-counter"; + public static final String ZOOKEEPER_SESSION_TIMEOUT = "ha.zookeeper.client.session-timeout"; public static final String ZOOKEEPER_CONNECTION_TIMEOUT = "ha.zookeeper.client.connection-timeout"; @@ -712,6 +727,12 @@ public final class ConfigConstants { public static final String DEFAULT_ZOOKEEPER_LEADER_PATH = "/leader"; + public static final String DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH = "/jobgraphs"; + + public static final String DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH = "/checkpoints"; + + public static final String DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "/checkpoint-counter"; + public static final int DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = 60000; public static final int DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT = 15000; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 8f0b19b2a0d6e..fdb59d9a89a56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -21,16 +21,16 @@ import akka.actor.ActorSystem; import akka.actor.PoisonPill; import akka.actor.Props; - import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.instance.AkkaActorGateway; import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; @@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; @@ -48,13 +47,19 @@ import java.util.TimerTask; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; /** * The checkpoint coordinator coordinates the distributed snapshots of operators and state. * It triggers the checkpoint by sending the messages to the relevant tasks and collects the * checkpoint acknowledgements. It also collects and maintains the overview of the state handles * reported by the tasks that acknowledge the checkpoint. + * + *

Depending on the configured {@link RecoveryMode}, the behaviour of the {@link + * CompletedCheckpointStore} and {@link CheckpointIDCounter} change. The default standalone + * implementations don't support any recovery. */ public class CheckpointCoordinator { @@ -79,12 +84,20 @@ public class CheckpointCoordinator { private final ExecutionVertex[] tasksToCommitTo; private final Map pendingCheckpoints; - - private final ArrayDeque completedCheckpoints; + + /** + * Completed checkpoints. Implementations can be blocking. Make sure calls to methods + * accessing this don't block the job manager actor and run asynchronously. + */ + private final CompletedCheckpointStore completedCheckpointStore; private final ArrayDeque recentPendingCheckpoints; - private final AtomicLong checkpointIdCounter = new AtomicLong(1); + /** + * Checkpoint ID counter to ensure ascending IDs. In case of job manager failures, these + * need to be ascending across job managers. + */ + private final CheckpointIDCounter checkpointIdCounter; private final AtomicInteger numUnsuccessfulCheckpointsTriggers = new AtomicInteger(); @@ -93,8 +106,6 @@ public class CheckpointCoordinator { private final long checkpointTimeout; - private final int numSuccessfulCheckpointsToRetain; - private TimerTask periodicScheduler; private ActorGateway jobStatusListener; @@ -110,61 +121,62 @@ public class CheckpointCoordinator { public CheckpointCoordinator( JobID job, - int numSuccessfulCheckpointsToRetain, long checkpointTimeout, ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, - ClassLoader userClassLoader) { + ClassLoader userClassLoader, + CheckpointIDCounter checkpointIDCounter, + CompletedCheckpointStore completedCheckpointStore, + RecoveryMode recoveryMode) throws Exception { - // some sanity checks - if (job == null || tasksToTrigger == null || - tasksToWaitFor == null || tasksToCommitTo == null) { - throw new NullPointerException(); - } - if (numSuccessfulCheckpointsToRetain < 1) { - throw new IllegalArgumentException("Must retain at least one successful checkpoint"); - } - if (checkpointTimeout < 1) { - throw new IllegalArgumentException("Checkpoint timeout must be larger than zero"); - } + // Sanity check + checkArgument(checkpointTimeout >= 1, "Checkpoint timeout must be larger than zero"); - this.job = job; - this.numSuccessfulCheckpointsToRetain = numSuccessfulCheckpointsToRetain; + this.job = checkNotNull(job); this.checkpointTimeout = checkpointTimeout; - this.tasksToTrigger = tasksToTrigger; - this.tasksToWaitFor = tasksToWaitFor; - this.tasksToCommitTo = tasksToCommitTo; + this.tasksToTrigger = checkNotNull(tasksToTrigger); + this.tasksToWaitFor = checkNotNull(tasksToWaitFor); + this.tasksToCommitTo = checkNotNull(tasksToCommitTo); this.pendingCheckpoints = new LinkedHashMap(); - this.completedCheckpoints = new ArrayDeque(numSuccessfulCheckpointsToRetain + 1); + this.completedCheckpointStore = checkNotNull(completedCheckpointStore); this.recentPendingCheckpoints = new ArrayDeque(NUM_GHOST_CHECKPOINT_IDS); this.userClassLoader = userClassLoader; + this.checkpointIdCounter = checkNotNull(checkpointIDCounter); + checkpointIDCounter.start(); - timer = new Timer("Checkpoint Timer", true); + this.timer = new Timer("Checkpoint Timer", true); - // Add shutdown hook to clean up state handles - shutdownHook = new Thread(new Runnable() { - @Override - public void run() { - try { - CheckpointCoordinator.this.shutdown(); - } - catch (Throwable t) { - LOG.error("Error during shutdown of blob service via JVM shutdown hook: " + - t.getMessage(), t); + if (recoveryMode == RecoveryMode.STANDALONE) { + // Add shutdown hook to clean up state handles when no checkpoint recovery is + // possible. In case of another configured recovery mode, the checkpoints need to be + // available for the standby job managers. + this.shutdownHook = new Thread(new Runnable() { + @Override + public void run() { + try { + CheckpointCoordinator.this.shutdown(); + } + catch (Throwable t) { + LOG.error("Error during shutdown of checkpoint coordniator via " + + "JVM shutdown hook: " + t.getMessage(), t); + } } - } - }); + }); - try { - // Add JVM shutdown hook to call shutdown of service - Runtime.getRuntime().addShutdownHook(shutdownHook); - } - catch (IllegalStateException ignored) { - // JVM is already shutting down. No need to do anything. + try { + // Add JVM shutdown hook to call shutdown of service + Runtime.getRuntime().addShutdownHook(shutdownHook); + } + catch (IllegalStateException ignored) { + // JVM is already shutting down. No need to do anything. + } + catch (Throwable t) { + LOG.error("Cannot register checkpoint coordinator shutdown hook.", t); + } } - catch (Throwable t) { - LOG.error("Cannot register checkpoint coordinator shutdown hook.", t); + else { + this.shutdownHook = null; } } @@ -178,41 +190,39 @@ public void run() { * After this method has been called, the coordinator does not accept and further * messages and cannot trigger any further checkpoints. */ - public void shutdown() { + public void shutdown() throws Exception { synchronized (lock) { - try { - if (shutdown) { - return; - } - shutdown = true; - LOG.info("Stopping checkpoint coordinator for job " + job); - - // shut down the thread that handles the timeouts - timer.cancel(); - - // make sure that the actor does not linger - if (jobStatusListener != null) { - jobStatusListener.tell(PoisonPill.getInstance()); - jobStatusListener = null; - } - - // the scheduling thread needs also to go away - if (periodicScheduler != null) { - periodicScheduler.cancel(); - periodicScheduler = null; - } - - // clear and discard all pending checkpoints - for (PendingCheckpoint pending : pendingCheckpoints.values()) { - pending.discard(userClassLoader, true); - } - pendingCheckpoints.clear(); - - // clean and discard all successful checkpoints - for (SuccessfulCheckpoint checkpoint : completedCheckpoints) { - checkpoint.discard(userClassLoader); + try { + if (!shutdown) { + shutdown = true; + LOG.info("Stopping checkpoint coordinator for job " + job); + + // shut down the thread that handles the timeouts + timer.cancel(); + + // make sure that the actor does not linger + if (jobStatusListener != null) { + jobStatusListener.tell(PoisonPill.getInstance()); + jobStatusListener = null; + } + + // the scheduling thread needs also to go away + if (periodicScheduler != null) { + periodicScheduler.cancel(); + periodicScheduler = null; + } + + checkpointIdCounter.stop(); + + // clear and discard all pending checkpoints + for (PendingCheckpoint pending : pendingCheckpoints.values()) { + pending.discard(userClassLoader, true); + } + pendingCheckpoints.clear(); + + // clean and discard all successful checkpoints + completedCheckpointStore.discardAllCheckpoints(); } - completedCheckpoints.clear(); } finally { // Remove shutdown hook to prevent resource leaks, unless this is invoked by the @@ -244,7 +254,7 @@ public boolean isShutdown() { * Triggers a new checkpoint and uses the current system time as the * checkpoint time. */ - public void triggerCheckpoint() { + public void triggerCheckpoint() throws Exception { triggerCheckpoint(System.currentTimeMillis()); } @@ -254,7 +264,7 @@ public void triggerCheckpoint() { * * @param timestamp The timestamp for the checkpoint. */ - public boolean triggerCheckpoint(final long timestamp) { + public boolean triggerCheckpoint(final long timestamp) throws Exception { if (shutdown) { LOG.error("Cannot trigger checkpoint, checkpoint coordinator has been shutdown."); return false; @@ -354,7 +364,7 @@ public void run() { } } - public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) { + public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws Exception { if (shutdown || message == null) { return; } @@ -365,7 +375,7 @@ public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) { final long checkpointId = message.getCheckpointId(); - SuccessfulCheckpoint completed = null; + CompletedCheckpoint completed = null; PendingCheckpoint checkpoint; synchronized (lock) { // we need to check inside the lock for being shutdown as well, otherwise we @@ -380,13 +390,13 @@ public void receiveAcknowledgeMessage(AcknowledgeCheckpoint message) { if (checkpoint.acknowledgeTask(message.getTaskExecutionId(), message.getState())) { if (checkpoint.isFullyAcknowledged()) { + completed = checkpoint.toCompletedCheckpoint(); + + completedCheckpointStore.addCheckpoint(completed); + LOG.info("Completed checkpoint " + checkpointId); + LOG.debug(completed.getStates().toString()); - completed = checkpoint.toCompletedCheckpoint(); - completedCheckpoints.addLast(completed); - if (completedCheckpoints.size() > numSuccessfulCheckpointsToRetain) { - completedCheckpoints.removeFirst().discard(userClassLoader); - } pendingCheckpoints.remove(checkpointId); rememberRecentCheckpointId(checkpointId); @@ -456,25 +466,30 @@ private void dropSubsumedCheckpoints(long timestamp) { // Checkpoint State Restoring // -------------------------------------------------------------------------------------------- - public void restoreLatestCheckpointedState(Map tasks, - boolean errorIfNoCheckpoint, - boolean allOrNothingState) throws Exception { + public void restoreLatestCheckpointedState( + Map tasks, + boolean errorIfNoCheckpoint, + boolean allOrNothingState) throws Exception { + synchronized (lock) { if (shutdown) { throw new IllegalStateException("CheckpointCoordinator is shut down"); } - - if (completedCheckpoints.isEmpty()) { + + // Recover the checkpoints + completedCheckpointStore.recover(); + + // restore from the latest checkpoint + CompletedCheckpoint latest = completedCheckpointStore.getLatestCheckpoint(); + + if (latest == null) { if (errorIfNoCheckpoint) { throw new IllegalStateException("No completed checkpoint available"); } else { return; } } - - // restore from the latest checkpoint - SuccessfulCheckpoint latest = completedCheckpoints.getLast(); - + if (allOrNothingState) { Map stateCounts = new HashMap(); @@ -519,7 +534,9 @@ public int getNumberOfPendingCheckpoints() { } public int getNumberOfRetainedSuccessfulCheckpoints() { - return this.completedCheckpoints.size(); + synchronized (lock) { + return completedCheckpointStore.getNumberOfRetainedCheckpoints(); + } } public Map getPendingCheckpoints() { @@ -528,9 +545,9 @@ public Map getPendingCheckpoints() { } } - public List getSuccessfulCheckpoints() { + public List getSuccessfulCheckpoints() throws Exception { synchronized (lock) { - return new ArrayList(this.completedCheckpoints); + return completedCheckpointStore.getAllCheckpoints(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java new file mode 100644 index 0000000000000..34b7946fc04c7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java @@ -0,0 +1,43 @@ +/* + * 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.checkpoint; + +/** + * A checkpoint ID counter. + */ +public interface CheckpointIDCounter { + + /** + * Starts the {@link CheckpointIDCounter} service. + */ + void start() throws Exception; + + /** + * Stops the {@link CheckpointIDCounter} service. + */ + void stop() throws Exception; + + /** + * Atomically increments the current checkpoint ID. + * + * @return The previous checkpoint ID + */ + long getAndIncrement() throws Exception; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java new file mode 100644 index 0000000000000..aa6e94b974df3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java @@ -0,0 +1,61 @@ +/* + * 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.checkpoint; + +import org.apache.flink.api.common.JobID; + +/** + * A factory for per Job checkpoint recovery components. + */ +public interface CheckpointRecoveryFactory { + + /** + * The number of {@link CompletedCheckpoint} instances to retain. + */ + int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1; + + /** + * Starts the {@link CheckpointRecoveryFactory} service. + */ + void start(); + + /** + * Stops the {@link CheckpointRecoveryFactory} service. + */ + void stop(); + + /** + * Creates a {@link CompletedCheckpointStore} instance for a job. + * + * @param jobId Job ID to recover checkpoints for + * @param userClassLoader User code class loader of the job + * @return {@link CompletedCheckpointStore} instance for the job + */ + CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader) + throws Exception; + + /** + * Creates a {@link CheckpointIDCounter} instance for a job. + * + * @param jobId Job ID to recover checkpoints for + * @return {@link CheckpointIDCounter} instance for the job + */ + CheckpointIDCounter createCheckpointIDCounter(JobID jobId) throws Exception; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java similarity index 85% rename from flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index be0b3010ec21c..ea3c26daae118 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SuccessfulCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -19,29 +19,28 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.io.Serializable; +import java.util.ArrayList; import java.util.List; /** * A successful checkpoint describes a checkpoint after all required tasks acknowledged it (with their state) * and that is considered completed. */ -public class SuccessfulCheckpoint { - - private static final Logger LOG = LoggerFactory.getLogger(SuccessfulCheckpoint.class); - +public class CompletedCheckpoint implements Serializable { + + private static final long serialVersionUID = -8360248179615702014L; + private final JobID job; private final long checkpointID; private final long timestamp; - private final List states; - + private final ArrayList states; - public SuccessfulCheckpoint(JobID job, long checkpointID, long timestamp, List states) { + public CompletedCheckpoint(JobID job, long checkpointID, long timestamp, ArrayList states) { this.job = job; this.checkpointID = checkpointID; this.timestamp = timestamp; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java new file mode 100644 index 0000000000000..d024aea5cf39f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java @@ -0,0 +1,69 @@ +/* + * 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.checkpoint; + +import java.util.List; + +/** + * A bounded LIFO-queue of {@link CompletedCheckpoint} instances. + */ +public interface CompletedCheckpointStore { + + /** + * Recover available {@link CompletedCheckpoint} instances. + * + *

After a call to this method, {@link #getLatestCheckpoint()} returns the latest + * available checkpoint. + */ + void recover() throws Exception; + + /** + * Adds a {@link CompletedCheckpoint} instance to the list of completed checkpoints. + * + *

Only a bounded number of checkpoints is kept. When exceeding the maximum number of + * retained checkpoints, the oldest one will be discarded via {@link + * CompletedCheckpoint#discard(ClassLoader)}. + */ + void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception; + + /** + * Returns the latest {@link CompletedCheckpoint} instance or null if none was + * added. + */ + CompletedCheckpoint getLatestCheckpoint() throws Exception; + + /** + * Discards all added {@link CompletedCheckpoint} instances via {@link + * CompletedCheckpoint#discard(ClassLoader)}. + */ + void discardAllCheckpoints() throws Exception; + + /** + * Returns all {@link CompletedCheckpoint} instances. + * + *

Returns an empty list if no checkpoint has been added yet. + */ + List getAllCheckpoints() throws Exception; + + /** + * Returns the current number of retained checkpoints. + */ + int getNumberOfRetainedCheckpoints(); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 370ae50fd0a2e..81159f602ce30 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -31,7 +31,7 @@ /** * A pending checkpoint is a checkpoint that has been started, but has not been * acknowledged by all tasks that need to acknowledge it. Once all tasks have - * acknowledged it, it becomes a {@link SuccessfulCheckpoint}. + * acknowledged it, it becomes a {@link CompletedCheckpoint}. * *

Note that the pending checkpoint, as well as the successful checkpoint keep the * state handles always as serialized values, never as actual values.

@@ -109,13 +109,13 @@ public List getCollectedStates() { return collectedStates; } - public SuccessfulCheckpoint toCompletedCheckpoint() { + public CompletedCheckpoint toCompletedCheckpoint() { synchronized (lock) { if (discarded) { throw new IllegalStateException("pending checkpoint is discarded"); } if (notYetAcknowledgedTasks.isEmpty()) { - SuccessfulCheckpoint completed = new SuccessfulCheckpoint(jobId, checkpointId, + CompletedCheckpoint completed = new CompletedCheckpoint(jobId, checkpointId, checkpointTimestamp, new ArrayList(collectedStates)); discard(null, false); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java new file mode 100644 index 0000000000000..052d743b5cbb2 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java @@ -0,0 +1,47 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.jobmanager.RecoveryMode; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * {@link CheckpointIDCounter} instances for JobManagers running in {@link RecoveryMode#STANDALONE}. + * + *

Simple wrapper of an {@link AtomicLong}. This is sufficient, because job managers are not + * recoverable in this recovery mode. + */ +public class StandaloneCheckpointIDCounter implements CheckpointIDCounter { + + private final AtomicLong checkpointIdCounter = new AtomicLong(1); + + @Override + public void start() throws Exception { + } + + @Override + public void stop() throws Exception { + } + + @Override + public long getAndIncrement() throws Exception { + return checkpointIdCounter.getAndIncrement(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java new file mode 100644 index 0000000000000..324a0be61a573 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java @@ -0,0 +1,52 @@ +/* + * 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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmanager.RecoveryMode; + +/** + * {@link CheckpointCoordinator} components in {@link RecoveryMode#STANDALONE}. + */ +public class StandaloneCheckpointRecoveryFactory implements CheckpointRecoveryFactory { + + @Override + public void start() { + // Nothing to do + } + + @Override + public void stop() { + // Nothing to do + } + + @Override + public CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader) + throws Exception { + + return new StandaloneCompletedCheckpointStore(CheckpointRecoveryFactory + .NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN, userClassLoader); + } + + @Override + public CheckpointIDCounter createCheckpointIDCounter(JobID ignored) { + return new StandaloneCheckpointIDCounter(); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java new file mode 100644 index 0000000000000..c31606a070c8f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java @@ -0,0 +1,100 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.jobmanager.RecoveryMode; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#STANDALONE}. + */ +class StandaloneCompletedCheckpointStore implements CompletedCheckpointStore { + + /** The maximum number of checkpoints to retain (at least 1). */ + private final int maxNumberOfCheckpointsToRetain; + + /** User class loader for discarding {@link CompletedCheckpoint} instances. */ + private final ClassLoader userClassLoader; + + /** The completed checkpoints. */ + private final ArrayDeque checkpoints; + + /** + * Creates {@link StandaloneCompletedCheckpointStore}. + * + * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at + * least 1). Adding more checkpoints than this results + * in older checkpoints being discarded. + * @param userClassLoader The user class loader used to discard checkpoints + */ + public StandaloneCompletedCheckpointStore( + int maxNumberOfCheckpointsToRetain, + ClassLoader userClassLoader) { + + checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint."); + + this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain; + this.userClassLoader = checkNotNull(userClassLoader, "User class loader"); + + this.checkpoints = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1); + } + + @Override + public void recover() throws Exception { + // Nothing to do + } + + @Override + public void addCheckpoint(CompletedCheckpoint checkpoint) { + checkpoints.addLast(checkpoint); + if (checkpoints.size() > maxNumberOfCheckpointsToRetain) { + checkpoints.removeFirst().discard(userClassLoader); + } + } + + @Override + public CompletedCheckpoint getLatestCheckpoint() { + return checkpoints.isEmpty() ? null : checkpoints.getLast(); + } + + @Override + public List getAllCheckpoints() { + return new ArrayList<>(checkpoints); + } + + @Override + public int getNumberOfRetainedCheckpoints() { + return checkpoints.size(); + } + + @Override + public void discardAllCheckpoints() { + for (CompletedCheckpoint checkpoint : checkpoints) { + checkpoint.discard(userClassLoader); + } + + checkpoints.clear(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java index 120c503e7bd06..d1428f4fde89d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateForTask.java @@ -24,6 +24,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + /** * Simple bean to describe the state belonging to a parallel operator. * Since we hold the state across execution attempts, we identify a task by its @@ -34,8 +39,10 @@ * Furthermore, the state may involve user-defined classes that are not accessible without * the respective classloader. */ -public class StateForTask { - +public class StateForTask implements Serializable { + + private static final long serialVersionUID = -2394696997971923995L; + private static final Logger LOG = LoggerFactory.getLogger(StateForTask.class); /** The state of the parallel operator */ @@ -48,12 +55,10 @@ public class StateForTask { private final int subtask; public StateForTask(SerializedValue> state, JobVertexID operatorId, int subtask) { - if (state == null || operatorId == null || subtask < 0) { - throw new IllegalArgumentException(); - } - - this.state = state; - this.operatorId = operatorId; + this.state = checkNotNull(state, "State"); + this.operatorId = checkNotNull(operatorId, "Operator ID"); + + checkArgument(subtask >= 0, "Negative subtask index"); this.subtask = subtask; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java new file mode 100644 index 0000000000000..66730508a67a1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java @@ -0,0 +1,130 @@ +/* + * 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.checkpoint; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.shared.SharedCount; +import org.apache.curator.framework.recipes.shared.VersionedValue; +import org.apache.curator.framework.state.ConnectionState; +import org.apache.curator.framework.state.ConnectionStateListener; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * {@link CheckpointIDCounter} instances for JobManagers running in {@link RecoveryMode#ZOOKEEPER}. + * + *

Each counter creates a ZNode: + *

+ * +----O /flink/checkpoint-counter/<job-id> 1 [persistent]
+ * .
+ * .
+ * .
+ * +----O /flink/checkpoint-counter/<job-id> N [persistent]
+ * 
+ * + *

The checkpoints IDs are required to be ascending (per job). In order to guarantee this in case + * of job manager failures we use ZooKeeper to have a shared counter across job manager instances. + */ +public class ZooKeeperCheckpointIDCounter implements CheckpointIDCounter { + + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperCheckpointIDCounter.class); + + /** Curator ZooKeeper client */ + private final CuratorFramework client; + + /** Path of the shared count */ + private final String counterPath; + + /** Curator recipe for shared counts */ + private final SharedCount sharedCount; + + /** Connection state listener to monitor the client connection */ + private final SharedCountConnectionStateListener connStateListener = + new SharedCountConnectionStateListener(); + + /** + * Creates a {@link ZooKeeperCheckpointIDCounter} instance. + * + * @param client Curator ZooKeeper client + * @param counterPath ZooKeeper path for the counter. It's sufficient to have a path per-job. + * @throws Exception + */ + public ZooKeeperCheckpointIDCounter(CuratorFramework client, String counterPath) throws Exception { + this.client = checkNotNull(client, "Curator client"); + this.counterPath = checkNotNull(counterPath, "Counter path"); + this.sharedCount = new SharedCount(client, counterPath, 1); + } + + @Override + public void start() throws Exception { + sharedCount.start(); + client.getConnectionStateListenable().addListener(connStateListener); + } + + @Override + public void stop() throws Exception { + sharedCount.close(); + client.getConnectionStateListenable().removeListener(connStateListener); + + LOG.info("Removing {} from ZooKeeper", counterPath); + client.delete().deletingChildrenIfNeeded().inBackground().forPath(counterPath); + } + + @Override + public long getAndIncrement() throws Exception { + while (true) { + ConnectionState connState = connStateListener.getLastState(); + + if (connState != null) { + throw new IllegalStateException("Connection state: " + connState); + } + + VersionedValue current = sharedCount.getVersionedValue(); + + Integer newCount = current.getValue() + 1; + + if (sharedCount.trySetCount(current, newCount)) { + return current.getValue(); + } + } + } + + /** + * Connection state listener. In case of {@link ConnectionState#SUSPENDED} or {@link + * ConnectionState#LOST} we are not guaranteed to read a current count from ZooKeeper. + */ + private class SharedCountConnectionStateListener implements ConnectionStateListener { + + private volatile ConnectionState lastState; + + @Override + public void stateChanged(CuratorFramework client, ConnectionState newState) { + if (newState == ConnectionState.SUSPENDED || newState == ConnectionState.LOST) { + lastState = newState; + } + } + + private ConnectionState getLastState() { + return lastState; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java new file mode 100644 index 0000000000000..2659e7e922228 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java @@ -0,0 +1,66 @@ +/* + * 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.checkpoint; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * {@link CheckpointCoordinator} components in {@link RecoveryMode#ZOOKEEPER}. + */ +public class ZooKeeperCheckpointRecoveryFactory implements CheckpointRecoveryFactory { + + private final CuratorFramework client; + + private final Configuration config; + + public ZooKeeperCheckpointRecoveryFactory(CuratorFramework client, Configuration config) { + this.client = checkNotNull(client, "Curator client"); + this.config = checkNotNull(config, "Configuration"); + } + + @Override + public void start() { + // Nothing to do + } + + @Override + public void stop() { + client.close(); + } + + @Override + public CompletedCheckpointStore createCompletedCheckpoints(JobID jobId, ClassLoader userClassLoader) + throws Exception { + + return ZooKeeperUtils.createCompletedCheckpoints(client, config, jobId, + NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN, userClassLoader); + } + + @Override + public CheckpointIDCounter createCheckpointIDCounter(JobID jobID) throws Exception { + return ZooKeeperUtils.createCheckpointIDCounter(client, config, jobID); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java new file mode 100644 index 0000000000000..62ab440ece0e5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -0,0 +1,293 @@ +/* + * 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.checkpoint; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.BackgroundCallback; +import org.apache.curator.framework.api.CuratorEvent; +import org.apache.curator.framework.api.CuratorEventType; +import org.apache.curator.utils.ZKPaths; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.ConcurrentModificationException; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * {@link CompletedCheckpointStore} for JobManagers running in {@link RecoveryMode#ZOOKEEPER}. + * + *

Checkpoints are added under a ZNode per job: + *

+ * +----O /flink/checkpoints/<job-id>  [persistent]
+ * .    |
+ * .    +----O /flink/checkpoints/<job-id>/1 [persistent]
+ * .    .                                  .
+ * .    .                                  .
+ * .    .                                  .
+ * .    +----O /flink/checkpoints/<job-id>/N [persistent]
+ * 
+ * + *

During recovery, the latest checkpoint is read from ZooKeeper. If there is more than one, + * only the latest one is used and older ones are discarded (even if the maximum number + * of retained checkpoints is greater than one). + * + *

If there is a network partition and multiple JobManagers run concurrent checkpoints for the + * same program, it is OK to take any valid successful checkpoint as long as the "history" of + * checkpoints is consistent. Currently, after recovery we start out with only a single + * checkpoint to circumvent those situations. + */ +public class ZooKeeperCompletedCheckpointStore implements CompletedCheckpointStore { + + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperCompletedCheckpointStore.class); + + /** Curator ZooKeeper client */ + private final CuratorFramework client; + + /** Completed checkpoints in ZooKeeper */ + private final ZooKeeperStateHandleStore checkpointsInZooKeeper; + + /** The maximum number of checkpoints to retain (at least 1). */ + private final int maxNumberOfCheckpointsToRetain; + + /** User class loader for discarding {@link CompletedCheckpoint} instances. */ + private final ClassLoader userClassLoader; + + /** Local completed checkpoints. */ + private final ArrayDeque, String>> checkpointStateHandles; + + /** + * Creates a {@link ZooKeeperCompletedCheckpointStore} instance. + * + * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at + * least 1). Adding more checkpoints than this results + * in older checkpoints being discarded. On recovery, + * we will only start with a single checkpoint. + * @param userClassLoader The user class loader used to discard checkpoints + * @param client The Curator ZooKeeper client + * @param checkpointsPath The ZooKeeper path for the checkpoints (needs to + * start with a '/') + * @param stateHandleProvider The state handle provider for checkpoints + * @throws Exception + */ + public ZooKeeperCompletedCheckpointStore( + int maxNumberOfCheckpointsToRetain, + ClassLoader userClassLoader, + CuratorFramework client, + String checkpointsPath, + StateHandleProvider stateHandleProvider) throws Exception { + + checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint."); + + this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain; + this.userClassLoader = checkNotNull(userClassLoader, "User class loader"); + + checkNotNull(client, "Curator client"); + checkNotNull(checkpointsPath, "Checkpoints path"); + checkNotNull(stateHandleProvider, "State handle provider"); + + // Ensure that the checkpoints path exists + client.newNamespaceAwareEnsurePath(checkpointsPath) + .ensure(client.getZookeeperClient()); + + // All operations will have the path as root + this.client = client.usingNamespace(client.getNamespace() + checkpointsPath); + + this.checkpointsInZooKeeper = new ZooKeeperStateHandleStore<>( + this.client, stateHandleProvider); + + this.checkpointStateHandles = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1); + + LOG.info("Initialized in '{}'.", checkpointsPath); + } + + /** + * Gets the latest checkpoint from ZooKeeper and removes all others. + * + *

Important: Even if there are more than one checkpoint in ZooKeeper, + * this will only recover the latest and discard the others. Otherwise, there is no guarantee + * that the history of checkpoints is consistent. + */ + @Override + public void recover() throws Exception { + LOG.info("Recovering checkpoints from ZooKeeper."); + + // Get all there is first + List, String>> initialCheckpoints; + while (true) { + try { + initialCheckpoints = checkpointsInZooKeeper.getAllSortedByName(); + break; + } + catch (ConcurrentModificationException e) { + LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying."); + } + } + + int numberOfInitialCheckpoints = initialCheckpoints.size(); + + LOG.info("Found {} checkpoints in ZooKeeper.", numberOfInitialCheckpoints); + + if (numberOfInitialCheckpoints > 0) { + // Take the last one. This is the latest checkpoints, because path names are strictly + // increasing (checkpoint ID). + Tuple2, String> latest = initialCheckpoints + .get(numberOfInitialCheckpoints - 1); + + CompletedCheckpoint latestCheckpoint = latest.f0.getState(userClassLoader); + + checkpointStateHandles.add(latest); + + LOG.info("Initialized with {}. Removing all older checkpoints.", latestCheckpoint); + + for (int i = 0; i < numberOfInitialCheckpoints - 1; i++) { + try { + removeFromZooKeeperAndDiscardCheckpoint(initialCheckpoints.get(i)); + } + catch (Exception e) { + LOG.error("Failed to discard checkpoint", e); + } + } + } + } + + /** + * Synchronously writes the new checkpoints to ZooKeeper and asynchronously removes older ones. + * + * @param checkpoint Completed checkpoint to add. + */ + @Override + public void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception { + checkNotNull(checkpoint, "Checkpoint"); + + // First add the new one. If it fails, we don't want to loose existing data. + String path = String.format("/%s", checkpoint.getCheckpointID()); + + final StateHandle stateHandle = checkpointsInZooKeeper.add(path, checkpoint); + + checkpointStateHandles.addLast(new Tuple2<>(stateHandle, path)); + + // Everything worked, let's remove a previous checkpoint if necessary. + if (checkpointStateHandles.size() > maxNumberOfCheckpointsToRetain) { + removeFromZooKeeperAndDiscardCheckpoint(checkpointStateHandles.removeFirst()); + } + + LOG.debug("Added {} to {}.", checkpoint, path); + } + + @Override + public CompletedCheckpoint getLatestCheckpoint() throws Exception { + if (checkpointStateHandles.isEmpty()) { + return null; + } + else { + return checkpointStateHandles.getLast().f0.getState(userClassLoader); + } + } + + @Override + public List getAllCheckpoints() throws Exception { + List checkpoints = new ArrayList<>(checkpointStateHandles.size()); + + for (Tuple2, String> stateHandle : checkpointStateHandles) { + checkpoints.add(stateHandle.f0.getState(userClassLoader)); + } + + return checkpoints; + } + + @Override + public int getNumberOfRetainedCheckpoints() { + return checkpointStateHandles.size(); + } + + @Override + public void discardAllCheckpoints() throws Exception { + for (Tuple2, String> checkpoint : checkpointStateHandles) { + try { + removeFromZooKeeperAndDiscardCheckpoint(checkpoint); + } + catch (Exception e) { + LOG.error("Failed to discard checkpoint.", e); + } + } + + checkpointStateHandles.clear(); + + String path = "/" + client.getNamespace(); + + LOG.info("Removing {} from ZooKeeper", path); + ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, true); + } + + /** + * Removes the state handle from ZooKeeper, discards the checkpoints, and the state handle. + */ + private void removeFromZooKeeperAndDiscardCheckpoint( + final Tuple2, String> stateHandleAndPath) throws Exception { + + final BackgroundCallback callback = new BackgroundCallback() { + @Override + public void processResult(CuratorFramework client, CuratorEvent event) throws Exception { + try { + if (event.getType() == CuratorEventType.DELETE) { + if (event.getResultCode() == 0) { + // The checkpoint + CompletedCheckpoint checkpoint = stateHandleAndPath + .f0.getState(userClassLoader); + + checkpoint.discard(userClassLoader); + + // Discard the state handle + stateHandleAndPath.f0.discardState(); + + // Discard the checkpoint + LOG.debug("Discarded " + checkpoint); + } + else { + throw new IllegalStateException("Unexpected result code " + + event.getResultCode() + " in '" + event + "' callback."); + } + } + else { + throw new IllegalStateException("Unexpected event type " + + event.getType() + " in '" + event + "' callback."); + } + } + catch (Exception e) { + LOG.error("Failed to discard checkpoint.", e); + } + } + }; + + // Remove state handle from ZooKeeper first. If this fails, we can still recover, but if + // we remove a state handle and fail to remove it from ZooKeeper, we end up in an + // inconsistent state. + checkpointsInZooKeeper.remove(stateHandleAndPath.f1, callback); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index ef004843d8fe8..9430d802d48df 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -30,6 +30,8 @@ import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -39,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; +import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.messages.ExecutionGraphMessages; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -110,8 +113,6 @@ public class ExecutionGraph implements Serializable { /** The log object used for debugging. */ static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class); - private static final int NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN = 1; - // -------------------------------------------------------------------------------------------- /** The lock used to secure all access to mutable fields, especially the tracking of progress @@ -347,7 +348,11 @@ public void enableSnapshotCheckpointing( List verticesToWaitFor, List verticesToCommitTo, ActorSystem actorSystem, - UUID leaderSessionID) { + UUID leaderSessionID, + CheckpointIDCounter checkpointIDCounter, + CompletedCheckpointStore completedCheckpointStore, + RecoveryMode recoveryMode) throws Exception { + // simple sanity checks if (interval < 10 || checkpointTimeout < 10) { throw new IllegalArgumentException(); @@ -367,12 +372,14 @@ public void enableSnapshotCheckpointing( snapshotCheckpointsEnabled = true; checkpointCoordinator = new CheckpointCoordinator( jobID, - NUMBER_OF_SUCCESSFUL_CHECKPOINTS_TO_RETAIN, checkpointTimeout, tasksToTrigger, tasksToWaitFor, tasksToCommitTo, - userClassLoader); + userClassLoader, + checkpointIDCounter, + completedCheckpointStore, + recoveryMode); // the periodic checkpoint scheduler is activated and deactivated as a result of // job status changes (running -> on, all other states -> off) @@ -382,8 +389,14 @@ public void enableSnapshotCheckpointing( interval, leaderSessionID)); } - - public void disableSnaphotCheckpointing() { + + /** + * Disables checkpointing. + * + *

The shutdown of the checkpoint coordinator might block. Make sure that calls to this + * method don't block the job manager actor and run asynchronously. + */ + public void disableSnaphotCheckpointing() throws Exception { if (state != JobStatus.CREATED) { throw new IllegalStateException("Job must be in CREATED state"); } @@ -772,6 +785,20 @@ public void restart() { } } + /** + * Restores the latest checkpointed state. + * + *

The recovery of checkpoints might block. Make sure that calls to this method don't + * block the job manager actor and run asynchronously. + */ + public void restoreLatestCheckpointedState() throws Exception { + synchronized (progressLock) { + if (checkpointCoordinator != null) { + checkpointCoordinator.restoreLatestCheckpointedState(getAllVertices(), false, false); + } + } + } + /** * This method cleans fields that are irrelevant for the archived execution attempt. */ @@ -886,7 +913,13 @@ public Object call() throws Exception { } }, executionContext); } else { - restart(); + future(new Callable() { + @Override + public Object call() throws Exception { + restart(); + return null; + } + }, executionContext); } break; } @@ -906,7 +939,7 @@ else if (numberOfRetriesLeft <= 0 && transitionState(current, JobStatus.FAILED, } } } - + private void postRunCleanup() { try { CheckpointCoordinator coord = this.checkpointCoordinator; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 6b36e2d374633..a64d63cc7008a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -538,4 +538,9 @@ public void uploadRequiredJarFiles(InetSocketAddress serverAddress) throws IOExc } } } + + @Override + public String toString() { + return "JobGraph(jobId: " + jobID + ")"; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java index 2e75b191fbe5e..17322d8f248af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java @@ -34,6 +34,19 @@ public enum RecoveryMode { STANDALONE, ZOOKEEPER; + /** + * Return the configured {@link RecoveryMode}. + * + * @param config The config to parse + * @return Configured recovery mode or {@link ConfigConstants#DEFAULT_RECOVERY_MODE} if not + * configured. + */ + public static RecoveryMode fromConfig(Configuration config) { + return RecoveryMode.valueOf(config.getString( + ConfigConstants.RECOVERY_MODE, + ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase()); + } + /** * Returns true if the defined recovery mode supports high availability. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java new file mode 100644 index 0000000000000..db36f9221c2cb --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStore.java @@ -0,0 +1,65 @@ +/* + * 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.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import scala.Option; + +import java.util.Collections; +import java.util.List; + +/** + * {@link SubmittedJobGraph} instances for JobManagers running in {@link RecoveryMode#STANDALONE}. + * + *

All operations are NoOps, because {@link JobGraph} instances cannot be recovered in this + * recovery mode. + */ +public class StandaloneSubmittedJobGraphStore implements SubmittedJobGraphStore { + + @Override + public void start(SubmittedJobGraphListener jobGraphListener) throws Exception { + // Nothing to do + } + + @Override + public void stop() { + // Nothing to do + } + + @Override + public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception { + // Nothing to do + } + + @Override + public void removeJobGraph(JobID jobId) throws Exception { + // Nothing to do + } + + @Override + public Option recoverJobGraph(JobID jobId) throws Exception { + return Option.empty(); + } + + @Override + public List recoverJobGraphs() throws Exception { + return Collections.emptyList(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java new file mode 100644 index 0000000000000..48da3b8ef8985 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraph.java @@ -0,0 +1,77 @@ +/* + * 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.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import java.io.Serializable; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A recoverable {@link JobGraph} and {@link JobInfo}. + */ +public class SubmittedJobGraph implements Serializable { + + private static final long serialVersionUID = 2836099271734771825L; + + /** The submitted {@link JobGraph} */ + private final JobGraph jobGraph; + + /** The {@link JobInfo}. */ + private final JobInfo jobInfo; + + /** + * Creates a {@link SubmittedJobGraph}. + * + * @param jobGraph The submitted {@link JobGraph} + * @param jobInfo The {@link JobInfo} + */ + public SubmittedJobGraph(JobGraph jobGraph, JobInfo jobInfo) { + this.jobGraph = checkNotNull(jobGraph, "Job graph"); + this.jobInfo = checkNotNull(jobInfo, "Job info"); + } + + /** + * Returns the submitted {@link JobGraph}. + */ + public JobGraph getJobGraph() { + return jobGraph; + } + + /** + * Returns the {@link JobID} of the submitted {@link JobGraph}. + */ + public JobID getJobId() { + return jobGraph.getJobID(); + } + + /** + * Returns the {@link JobInfo} of the client who submitted the {@link JobGraph}. + */ + public JobInfo getJobInfo() throws Exception { + return jobInfo; + } + + @Override + public String toString() { + return String.format("SubmittedJobGraph(%s, %s)", jobGraph, jobInfo); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java new file mode 100644 index 0000000000000..bd628cd433c90 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/SubmittedJobGraphStore.java @@ -0,0 +1,93 @@ +/* + * 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.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import scala.Option; + +import java.util.List; + +/** + * {@link SubmittedJobGraph} instances for recovery. + */ +public interface SubmittedJobGraphStore { + + /** + * Starts the {@link SubmittedJobGraphStore} service. + */ + void start(SubmittedJobGraphListener jobGraphListener) throws Exception; + + /** + * Stops the {@link SubmittedJobGraphStore} service. + */ + void stop() throws Exception; + + /** + * Returns a list of all submitted {@link JobGraph} instances. + */ + List recoverJobGraphs() throws Exception; + + /** + * Returns the {@link SubmittedJobGraph} with the given {@link JobID}. + * + *

An Exception is thrown, if no job graph with the given ID exists. + */ + Option recoverJobGraph(JobID jobId) throws Exception; + + /** + * Adds the {@link SubmittedJobGraph} instance. + * + *

If a job graph with the same {@link JobID} exists, it is replaced. + */ + void putJobGraph(SubmittedJobGraph jobGraph) throws Exception; + + /** + * Removes the {@link SubmittedJobGraph} with the given {@link JobID} if it exists. + */ + void removeJobGraph(JobID jobId) throws Exception; + + /** + * A listener for {@link SubmittedJobGraph} instances. This is used to react to races between + * multiple running {@link SubmittedJobGraphStore} instances (on multiple job managers). + */ + interface SubmittedJobGraphListener { + + /** + * Callback for {@link SubmittedJobGraph} instances added by a different {@link + * SubmittedJobGraphStore} instance. + * + *

Important: It is possible to get false positives and be notified + * about a job graph, which was added by this instance. + * + * @param jobId The {@link JobID} of the added job graph + */ + void onAddedJobGraph(JobID jobId); + + /** + * Callback for {@link SubmittedJobGraph} instances removed by a different {@link + * SubmittedJobGraphStore} instance. + * + * @param jobId The {@link JobID} of the removed job graph + */ + void onRemovedJobGraph(JobID jobId); + + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java new file mode 100644 index 0000000000000..660f8bc852eaa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphStore.java @@ -0,0 +1,379 @@ +/* + * 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.jobmanager; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.utils.ZKPaths; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +/** + * {@link SubmittedJobGraph} instances for JobManagers running in {@link RecoveryMode#ZOOKEEPER}. + * + *

Each job graph creates ZNode: + *

+ * +----O /flink/jobgraphs/<job-id> 1 [persistent]
+ * .
+ * .
+ * .
+ * +----O /flink/jobgraphs/<job-id> N [persistent]
+ * 
+ * + *

The root path is watched to detect concurrent modifications in corner situations where + * multiple instances operate concurrently. The job manager acts as a {@link SubmittedJobGraphListener} + * to react to such situations. + */ +public class ZooKeeperSubmittedJobGraphStore implements SubmittedJobGraphStore { + + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperSubmittedJobGraphStore.class); + + /** Lock to synchronize with the {@link SubmittedJobGraphListener}. */ + private final Object cacheLock = new Object(); + + /** Client (not a namespace facade) */ + private final CuratorFramework client; + + /** The set of IDs of all added job graphs. */ + private final Set addedJobGraphs = new HashSet<>(); + + /** Completed checkpoints in ZooKeeper */ + private final ZooKeeperStateHandleStore jobGraphsInZooKeeper; + + /** + * Cache to monitor all children. This is used to detect races with other instances working + * on the same state. + */ + private final PathChildrenCache pathCache; + + /** The external listener to be notified on races. */ + private SubmittedJobGraphListener jobGraphListener; + + /** Flag indicating whether this instance is running. */ + private boolean isRunning; + + public ZooKeeperSubmittedJobGraphStore( + CuratorFramework client, + String currentJobsPath, + StateHandleProvider stateHandleProvider) throws Exception { + + checkNotNull(currentJobsPath, "Current jobs path"); + checkNotNull(stateHandleProvider, "State handle provider"); + + // Keep a reference to the original client and not the namespace facade. The namespace + // facade cannot be closed. + this.client = checkNotNull(client, "Curator client"); + + // Ensure that the job graphs path exists + client.newNamespaceAwareEnsurePath(currentJobsPath) + .ensure(client.getZookeeperClient()); + + // All operations will have the path as root + client = client.usingNamespace(client.getNamespace() + currentJobsPath); + + this.jobGraphsInZooKeeper = new ZooKeeperStateHandleStore<>(client, stateHandleProvider); + + this.pathCache = new PathChildrenCache(client, "/", false); + pathCache.getListenable().addListener(new SubmittedJobGraphsPathCacheListener()); + } + + @Override + public void start(SubmittedJobGraphListener jobGraphListener) throws Exception { + synchronized (cacheLock) { + if (!isRunning) { + this.jobGraphListener = jobGraphListener; + + pathCache.start(); + + isRunning = true; + } + } + } + + @Override + public void stop() throws Exception { + synchronized (cacheLock) { + if (isRunning) { + jobGraphListener = null; + + pathCache.close(); + + client.close(); + + isRunning = false; + } + } + } + + @Override + public List recoverJobGraphs() throws Exception { + synchronized (cacheLock) { + verifyIsRunning(); + + List, String>> submitted; + + while (true) { + try { + submitted = jobGraphsInZooKeeper.getAll(); + break; + } + catch (ConcurrentModificationException e) { + LOG.warn("Concurrent modification while reading from ZooKeeper. Retrying."); + } + } + + if (submitted.size() != 0) { + List jobGraphs = new ArrayList<>(submitted.size()); + + for (Tuple2, String> jobStateHandle : submitted) { + SubmittedJobGraph jobGraph = jobStateHandle + .f0.getState(ClassLoader.getSystemClassLoader()); + + addedJobGraphs.add(jobGraph.getJobId()); + + jobGraphs.add(jobGraph); + } + + LOG.info("Recovered {} job graphs: {}.", jobGraphs.size(), jobGraphs); + return jobGraphs; + } + else { + LOG.info("No job graph to recover."); + return Collections.emptyList(); + } + } + } + + @Override + public Option recoverJobGraph(JobID jobId) throws Exception { + checkNotNull(jobId, "Job ID"); + String path = getPathForJob(jobId); + + synchronized (cacheLock) { + verifyIsRunning(); + + try { + StateHandle jobStateHandle = jobGraphsInZooKeeper.get(path); + + SubmittedJobGraph jobGraph = jobStateHandle + .getState(ClassLoader.getSystemClassLoader()); + + addedJobGraphs.add(jobGraph.getJobId()); + + LOG.info("Recovered {}.", jobGraph); + + return Option.apply(jobGraph); + } + catch (KeeperException.NoNodeException ignored) { + return Option.empty(); + } + } + } + + @Override + public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception { + checkNotNull(jobGraph, "Job graph"); + String path = getPathForJob(jobGraph.getJobId()); + + boolean success = false; + + while (!success) { + synchronized (cacheLock) { + verifyIsRunning(); + + int currentVersion = jobGraphsInZooKeeper.exists(path); + + if (currentVersion == -1) { + try { + jobGraphsInZooKeeper.add(path, jobGraph); + + addedJobGraphs.add(jobGraph.getJobId()); + + LOG.info("Added {} to ZooKeeper.", jobGraph); + + success = true; + } + catch (KeeperException.NodeExistsException ignored) { + } + } + else if (addedJobGraphs.contains(jobGraph.getJobId())) { + try { + jobGraphsInZooKeeper.replace(path, currentVersion, jobGraph); + LOG.info("Updated {} in ZooKeeper.", jobGraph); + + success = true; + } + catch (KeeperException.NoNodeException ignored) { + } + } + else { + throw new IllegalStateException("Oh, no. Trying to update a graph you didn't " + + "#getAllSubmittedJobGraphs() or #putJobGraph() yourself before."); + } + } + } + } + + @Override + public void removeJobGraph(JobID jobId) throws Exception { + checkNotNull(jobId, "Job ID"); + String path = getPathForJob(jobId); + + synchronized (cacheLock) { + if (addedJobGraphs.contains(jobId)) { + jobGraphsInZooKeeper.removeAndDiscardState(path); + + addedJobGraphs.remove(jobId); + LOG.info("Removed job graph {} from ZooKeeper.", jobId); + } + } + } + + /** + * Monitors ZooKeeper for changes. + * + *

Detects modifications from other job managers in corner situations. The event + * notifications fire for changes from this job manager as well. + */ + private final class SubmittedJobGraphsPathCacheListener implements PathChildrenCacheListener { + + @Override + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) + throws Exception { + + if (LOG.isDebugEnabled()) { + if (event.getData() != null) { + LOG.debug("Received {} event (path: {})", event.getType(), event.getData().getPath()); + } + else { + LOG.debug("Received {} event", event.getType()); + } + } + + switch (event.getType()) { + case CHILD_ADDED: + synchronized (cacheLock) { + try { + JobID jobId = fromEvent(event); + if (jobGraphListener != null && !addedJobGraphs.contains(jobId)) { + try { + // Whoa! This has been added by someone else. Or we were fast + // to remove it (false positive). + jobGraphListener.onAddedJobGraph(jobId); + } + catch (Throwable t) { + LOG.error("Error in callback", t); + } + } + } + catch (Exception e) { + LOG.error("Error in SubmittedJobGraphsPathCacheListener", e); + } + } + + break; + + case CHILD_UPDATED: + // Nothing to do + break; + + case CHILD_REMOVED: + synchronized (cacheLock) { + try { + JobID jobId = fromEvent(event); + if (jobGraphListener != null && addedJobGraphs.contains(jobId)) { + try { + // Oh oh. Someone else removed one of our job graphs. Mean! + jobGraphListener.onRemovedJobGraph(jobId); + } + catch (Throwable t) { + LOG.error("Error in callback", t); + } + } + + break; + } + catch (Exception e) { + LOG.error("Error in SubmittedJobGraphsPathCacheListener", e); + } + } + break; + + case CONNECTION_SUSPENDED: + LOG.warn("ZooKeeper connection SUSPENDED. Changes to the submitted job " + + "graphs are not monitored (temporarily)."); + + case CONNECTION_LOST: + LOG.warn("ZooKeeper connection LOST. Changes to the submitted job " + + "graphs are not monitored (permanently)."); + break; + + case CONNECTION_RECONNECTED: + LOG.info("ZooKeeper connection RECONNECTED. Changes to the submitted job " + + "graphs are monitored again."); + + case INITIALIZED: + LOG.info("SubmittedJobGraphsPathCacheListener initialized"); + break; + } + } + + /** + * Returns a JobID for the event's path. + */ + private JobID fromEvent(PathChildrenCacheEvent event) { + return JobID.fromHexString(ZKPaths.getNodeFromPath(event.getData().getPath())); + } + } + + /** + * Verifies that the state is running. + */ + private void verifyIsRunning() { + checkState(isRunning, "Not running. Forgot to call start()?"); + } + + /** + * Returns the JobID as a String (with leading slash). + */ + public static String getPathForJob(JobID jobId) { + checkNotNull(jobId, "Job ID"); + return String.format("/%s", jobId); + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java index b6223eeb36812..6cba1410f0a07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/LeaderElectionService.java @@ -67,4 +67,5 @@ public interface LeaderElectionService { * @return true if the associated {@link LeaderContender} is the leader, otherwise false */ boolean hasLeadership(); + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java index ae3f0e6e380e2..811037c9a9761 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java @@ -43,6 +43,7 @@ * ZooKeeper as well. */ public class ZooKeeperLeaderElectionService implements LeaderElectionService, LeaderLatchListener, NodeCacheListener { + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperLeaderElectionService.class); /** Client to the ZooKeeper quorum */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java new file mode 100644 index 0000000000000..7aa1ccfe329e7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java @@ -0,0 +1,39 @@ +/* + * 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.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; + +public enum StateBackend { + JOBMANAGER, FILESYSTEM; + + /** + * Returns the configured {@link StateBackend}. + * + * @param config The config to parse + * @return Configured state backend or {@link ConfigConstants#DEFAULT_RECOVERY_MODE} if not + * configured. + */ + public static StateBackend fromConfig(Configuration config) { + return StateBackend.valueOf(config.getString( + ConfigConstants.STATE_BACKEND, + ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java new file mode 100644 index 0000000000000..0086ac6f5158f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProviderFactory.java @@ -0,0 +1,61 @@ +/* + * 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.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; + +import java.io.Serializable; + +/** + * State handler provider factory. + * + *

This is going to be superseded soon. + */ +public class StateHandleProviderFactory { + + /** + * Creates a {@link org.apache.flink.runtime.state.FileStateHandle.FileStateHandleProvider} at + * the configured recovery path. + */ + public static StateHandleProvider createRecoveryFileStateHandleProvider( + Configuration config) { + + StateBackend stateBackend = StateBackend.fromConfig(config); + + if (stateBackend == StateBackend.FILESYSTEM) { + String recoveryPath = config.getString( + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, ""); + + if (recoveryPath.equals("")) { + throw new IllegalConfigurationException("Missing recovery path. Specify via " + + "configuration key '" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "'."); + } + else { + return FileStateHandle.createProvider(recoveryPath); + } + } + else { + throw new IllegalConfigurationException("Unexpected state backend configuration " + + stateBackend); + } + } + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java deleted file mode 100644 index 5f867a51e1c1e..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/LeaderElectionUtils.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.util; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobmanager.RecoveryMode; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService; - -/** - * Utility class to help working with {@link LeaderElectionService} class. - */ -public final class LeaderElectionUtils { - - /** - * Creates a {@link LeaderElectionService} based on the provided {@link Configuration} object. - * - * @param configuration Configuration object - * @return {@link LeaderElectionService} which was created based on the provided Configuration - * @throws Exception - */ - public static LeaderElectionService createLeaderElectionService(Configuration configuration) throws Exception { - RecoveryMode recoveryMode = RecoveryMode.valueOf(configuration.getString( - ConfigConstants.RECOVERY_MODE, - ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase() - ); - - LeaderElectionService leaderElectionService; - - switch(recoveryMode) { - case STANDALONE: - leaderElectionService = new StandaloneLeaderElectionService(); - break; - case ZOOKEEPER: - leaderElectionService = ZooKeeperUtils.createLeaderElectionService(configuration); - break; - default: - throw new Exception("Unknown RecoveryMode " + recoveryMode); - } - - return leaderElectionService; - } - - /** - * Private constructor to prevent instantiation. - */ - private LeaderElectionUtils() { - throw new RuntimeException(); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index d2d3945ce5f55..79b9b7e345c85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -21,19 +21,27 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore; import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraph; +import org.apache.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore; import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService; +import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.runtime.state.StateHandleProviderFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Utility class to work with Apache Zookeeper for Flink runtime. - */ -public final class ZooKeeperUtils { +import static com.google.common.base.Preconditions.checkNotNull; + +public class ZooKeeperUtils { private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtils.class); @@ -47,8 +55,10 @@ public final class ZooKeeperUtils { public static CuratorFramework startCuratorFramework(Configuration configuration) { String zkQuorum = configuration.getString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, ""); - if(zkQuorum == null || zkQuorum.equals("")) { - throw new RuntimeException("No valid ZooKeeper quorum has been specified."); + if (zkQuorum == null || zkQuorum.equals("")) { + throw new RuntimeException("No valid ZooKeeper quorum has been specified. " + + "You can specify the quorum via the configuration key '" + + ConfigConstants.ZOOKEEPER_QUORUM_KEY + "'."); } int sessionTimeout = configuration.getInteger( @@ -59,7 +69,7 @@ public static CuratorFramework startCuratorFramework(Configuration configuration ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT, ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT); - int retryWait = configuration.getInteger ( + int retryWait = configuration.getInteger( ConfigConstants.ZOOKEEPER_RETRY_WAIT, ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT); @@ -88,14 +98,10 @@ public static CuratorFramework startCuratorFramework(Configuration configuration } /** - * Returns whether high availability is enabled (<=> ZooKeeper quorum configured). + * Returns whether {@link RecoveryMode#ZOOKEEPER} is configured. */ - public static boolean isZooKeeperHighAvailabilityEnabled(Configuration flinkConf) { - String recoveryMode = flinkConf.getString( - ConfigConstants.RECOVERY_MODE, - ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase(); - - return recoveryMode.equals(RecoveryMode.ZOOKEEPER.name()); + public static boolean isZooKeeperRecoveryMode(Configuration flinkConf) { + return RecoveryMode.fromConfig(flinkConf).equals(RecoveryMode.ZOOKEEPER); } /** @@ -125,7 +131,7 @@ public static String getZooKeeperEnsemble(Configuration flinkConf) * @throws Exception */ public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService( - Configuration configuration) throws Exception{ + Configuration configuration) throws Exception { CuratorFramework client = startCuratorFramework(configuration); String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH, ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PATH); @@ -134,7 +140,8 @@ public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService( } /** - * Creates a {@link ZooKeeperLeaderElectionService} instance. + * Creates a {@link ZooKeeperLeaderElectionService} instance and a new {@link + * CuratorFramework} client. * * @param configuration {@link Configuration} object containing the configuration values * @return {@link ZooKeeperLeaderElectionService} instance. @@ -142,8 +149,24 @@ public static ZooKeeperLeaderRetrievalService createLeaderRetrievalService( */ public static ZooKeeperLeaderElectionService createLeaderElectionService( Configuration configuration) throws Exception { + CuratorFramework client = startCuratorFramework(configuration); + return createLeaderElectionService(client, configuration); + } + + /** + * Creates a {@link ZooKeeperLeaderElectionService} instance. + * + * @param client The {@link CuratorFramework} ZooKeeper client to use + * @param configuration {@link Configuration} object containing the configuration values + * @return {@link ZooKeeperLeaderElectionService} instance. + * @throws Exception + */ + public static ZooKeeperLeaderElectionService createLeaderElectionService( + CuratorFramework client, + Configuration configuration) throws Exception { + String latchPath = configuration.getString(ConfigConstants.ZOOKEEPER_LATCH_PATH, ConfigConstants.DEFAULT_ZOOKEEPER_LATCH_PATH); String leaderPath = configuration.getString(ConfigConstants.ZOOKEEPER_LEADER_PATH, @@ -152,6 +175,89 @@ public static ZooKeeperLeaderElectionService createLeaderElectionService( return new ZooKeeperLeaderElectionService(client, latchPath, leaderPath); } + /** + * Creates a {@link ZooKeeperSubmittedJobGraphStore} instance. + * + * @param client The {@link CuratorFramework} ZooKeeper client to use + * @param configuration {@link Configuration} object + * @return {@link ZooKeeperSubmittedJobGraphStore} instance + */ + public static ZooKeeperSubmittedJobGraphStore createSubmittedJobGraphs( + CuratorFramework client, + Configuration configuration) throws Exception { + + checkNotNull(configuration, "Configuration"); + + StateHandleProvider stateHandleProvider = + StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration); + + // ZooKeeper submitted jobs root dir + String zooKeeperSubmittedJobsPath = configuration.getString( + ConfigConstants.ZOOKEEPER_JOBGRAPHS_PATH, + ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); + + return new ZooKeeperSubmittedJobGraphStore( + client, zooKeeperSubmittedJobsPath, stateHandleProvider); + } + + /** + * Creates a {@link ZooKeeperCompletedCheckpointStore} instance. + * + * @param client The {@link CuratorFramework} ZooKeeper client to use + * @param configuration {@link Configuration} object + * @param jobId ID of job to create the instance for + * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain + * @param userClassLoader User code class loader + * @return {@link ZooKeeperCompletedCheckpointStore} instance + */ + public static CompletedCheckpointStore createCompletedCheckpoints( + CuratorFramework client, + Configuration configuration, + JobID jobId, + int maxNumberOfCheckpointsToRetain, + ClassLoader userClassLoader) throws Exception { + + checkNotNull(configuration, "Configuration"); + + StateHandleProvider stateHandleProvider = + StateHandleProviderFactory.createRecoveryFileStateHandleProvider(configuration); + + String completedCheckpointsPath = configuration.getString( + ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); + + completedCheckpointsPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId); + + return new ZooKeeperCompletedCheckpointStore( + maxNumberOfCheckpointsToRetain, + userClassLoader, + client, + completedCheckpointsPath, + stateHandleProvider); + } + + /** + * Creates a {@link ZooKeeperCheckpointIDCounter} instance. + * + * @param client The {@link CuratorFramework} ZooKeeper client to use + * @param configuration {@link Configuration} object + * @param jobId ID of job to create the instance for + * @return {@link ZooKeeperCheckpointIDCounter} instance + */ + public static ZooKeeperCheckpointIDCounter createCheckpointIDCounter( + CuratorFramework client, + Configuration configuration, + JobID jobId) throws Exception { + + String checkpointIdCounterPath = configuration.getString( + ConfigConstants.ZOOKEEPER_CHECKPOINT_COUNTER_PATH, + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); + + checkpointIdCounterPath += ZooKeeperSubmittedJobGraphStore.getPathForJob(jobId); + + return new ZooKeeperCheckpointIDCounter(client, checkpointIdCounterPath); + } + /** * Private constructor to prevent instantiation. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java new file mode 100644 index 0000000000000..936fe1ba52b62 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java @@ -0,0 +1,384 @@ +/* + * 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.zookeeper; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.BackgroundCallback; +import org.apache.curator.utils.ZKPaths; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.util.InstantiationUtil; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * State handles backed by ZooKeeper. + * + *

Added state is persisted via {@link StateHandle}s, which in turn are written to + * ZooKeeper. This level of indirection is necessary to keep the amount of data in ZooKeeper + * small. ZooKeeper is build for data in the KB range whereas state can grow to multiple MBs. + * + *

State modifications require some care, because it is possible that certain failures bring + * the state handle backend and ZooKeeper out of sync. + * + *

ZooKeeper holds the ground truth about state handles, i.e. the following holds: + * + *

+ * State handle in ZooKeeper => State handle exists
+ * 
+ * + * But not: + * + *
+ * State handle exists => State handle in ZooKeeper
+ * 
+ * + * There can be lingering state handles when failures happen during operation. They + * need to be cleaned up manually (see + * FLINK-2513 about a possible way to overcome this). + * + * @param Type of state + */ +public class ZooKeeperStateHandleStore { + + /** Curator ZooKeeper client */ + private final CuratorFramework client; + + /** State handle provider */ + private final StateHandleProvider stateHandleProvider; + + /** + * Creates a {@link ZooKeeperStateHandleStore}. + * + * @param client The Curator ZooKeeper client. Important: It is + * expected that the client's namespace ensures that the root + * path is exclusive for all state handles managed by this + * instance, e.g. client.usingNamespace("/stateHandles") + * @param stateHandleProvider The state handle provider for the state + */ + public ZooKeeperStateHandleStore( + CuratorFramework client, + StateHandleProvider stateHandleProvider) { + + this.client = checkNotNull(client, "Curator client"); + this.stateHandleProvider = checkNotNull(stateHandleProvider, "State handle provider"); + } + + /** + * Creates a state handle and stores it in ZooKeeper with create mode {@link + * CreateMode#PERSISTENT}. + * + * @see #add(String, Serializable, CreateMode) + */ + public StateHandle add(String pathInZooKeeper, T state) throws Exception { + return add(pathInZooKeeper, state, CreateMode.PERSISTENT); + } + + /** + * Creates a state handle and stores it in ZooKeeper. + * + *

Important: This will not store the actual state in + * ZooKeeper, but create a state handle and store it in ZooKeeper. This level of indirection + * makes sure that data in ZooKeeper is small. + * + * @param pathInZooKeeper Destination path in ZooKeeper (expected to *not* exist yet and + * start with a '/') + * @param state State to be added + * @param createMode The create mode for the new path in ZooKeeper + * @return Created {@link StateHandle} + * @throws Exception If a ZooKeeper or state handle operation fails + */ + public StateHandle add(String pathInZooKeeper, T state, CreateMode createMode) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + checkNotNull(state, "State"); + + // Create the state handle. Nothing persisted yet. + StateHandle stateHandle = stateHandleProvider.createStateHandle(state); + + boolean success = false; + + try { + // Serialize the state handle. This writes the state to the backend. + byte[] serializedStateHandle = InstantiationUtil.serializeObject(stateHandle); + + // Write state handle (not the actual state) to ZooKeeper. This is expected to be + // smaller than the state itself. This level of indirection makes sure that data in + // ZooKeeper is small, because ZooKeeper is designed for data in the KB range, but + // the state can be larger. + client.create().withMode(createMode).forPath(pathInZooKeeper, serializedStateHandle); + + success = true; + + return stateHandle; + } + finally { + if (!success) { + // Cleanup the state handle if it was not written to ZooKeeper. + if (stateHandle != null) { + stateHandle.discardState(); + } + } + } + } + + /** + * Replaces a state handle in ZooKeeper and discards the old state handle. + * + * @param pathInZooKeeper Destination path in ZooKeeper (expected to exist and start with a '/') + * @param expectedVersion Expected version of the node to replace + * @param state The new state to replace the old one + * @throws Exception If a ZooKeeper or state handle operation fails + */ + public void replace(String pathInZooKeeper, int expectedVersion, T state) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + checkNotNull(state, "State"); + + StateHandle oldStateHandle = get(pathInZooKeeper); + + StateHandle stateHandle = stateHandleProvider.createStateHandle(state); + + boolean success = false; + + try { + // Serialize the new state handle. This writes the state to the backend. + byte[] serializedStateHandle = InstantiationUtil.serializeObject(stateHandle); + + // Replace state handle in ZooKeeper. + client.setData() + .withVersion(expectedVersion) + .forPath(pathInZooKeeper, serializedStateHandle); + + success = true; + } + finally { + if (success) { + oldStateHandle.discardState(); + } + else { + stateHandle.discardState(); + } + } + } + + /** + * Returns the version of the node if it exists or -1 if it doesn't. + * + * @param pathInZooKeeper Path in ZooKeeper to check + * @return Version of the ZNode if the path exists, -1 otherwise. + * @throws Exception If the ZooKeeper operation fails + */ + public int exists(String pathInZooKeeper) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + + Stat stat = client.checkExists().forPath(pathInZooKeeper); + + if (stat != null) { + return stat.getVersion(); + } + + return -1; + } + + /** + * Gets a state handle from ZooKeeper. + * + * @param pathInZooKeeper Path in ZooKeeper to get the state handle from (expected to + * exist and start with a '/'). + * @return The state handle + * @throws Exception If a ZooKeeper or state handle operation fails + */ + @SuppressWarnings("unchecked") + public StateHandle get(String pathInZooKeeper) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + + byte[] data = client.getData().forPath(pathInZooKeeper); + + return (StateHandle) InstantiationUtil + .deserializeObject(data, ClassLoader.getSystemClassLoader()); + } + + /** + * Gets all available state handles from ZooKeeper. + * + *

If there is a concurrent modification, the operation is retried until it succeeds. + * + * @return All state handles from ZooKeeper. + * @throws Exception If a ZooKeeper or state handle operation fails + */ + @SuppressWarnings("unchecked") + public List, String>> getAll() throws Exception { + final List, String>> stateHandles = new ArrayList<>(); + + boolean success = false; + + retry: + while (!success) { + // Initial cVersion (number of changes to the children of this node) + int initialCVersion = client.checkExists().forPath("/").getCversion(); + + List children = client.getChildren().forPath("/"); + + for (String path : children) { + path = "/" + path; + + try { + final StateHandle stateHandle = get(path); + stateHandles.add(new Tuple2<>(stateHandle, path)); + } + catch (KeeperException.NoNodeException ignored) { + // Concurrent deletion, retry + continue retry; + } + } + + int finalCVersion = client.checkExists().forPath("/").getCversion(); + + // Check for concurrent modifications + success = initialCVersion == finalCVersion; + } + + return stateHandles; + } + + /** + * Gets all available state handles from ZooKeeper sorted by name (ascending). + * + *

If there is a concurrent modification, the operation is retried until it succeeds. + * + * @return All state handles in ZooKeeper. + * @throws Exception If a ZooKeeper or state handle operation fails + */ + @SuppressWarnings("unchecked") + public List, String>> getAllSortedByName() throws Exception { + final List, String>> stateHandles = new ArrayList<>(); + + boolean success = false; + + retry: + while (!success) { + // Initial cVersion (number of changes to the children of this node) + int initialCVersion = client.checkExists().forPath("/").getCversion(); + + List children = ZKPaths.getSortedChildren( + client.getZookeeperClient().getZooKeeper(), + ZKPaths.fixForNamespace(client.getNamespace(), "/")); + + for (String path : children) { + path = "/" + path; + + try { + final StateHandle stateHandle = get(path); + stateHandles.add(new Tuple2<>(stateHandle, path)); + } + catch (KeeperException.NoNodeException ignored) { + // Concurrent deletion, retry + continue retry; + } + } + + int finalCVersion = client.checkExists().forPath("/").getCversion(); + + // Check for concurrent modifications + success = initialCVersion == finalCVersion; + } + + return stateHandles; + } + + /** + * Removes a state handle from ZooKeeper. + * + *

Important: this does not discard the state handle. If you want to + * discard the state handle call {@link #removeAndDiscardState(String)}. + * + * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/') + * @throws Exception If the ZooKeeper operation fails + */ + public void remove(String pathInZooKeeper) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + + client.delete().deletingChildrenIfNeeded().forPath(pathInZooKeeper); + } + + /** + * Removes a state handle from ZooKeeper asynchronously. + * + *

Important: this does not discard the state handle. If you want to + * discard the state handle call {@link #removeAndDiscardState(String)}. + * + * @param pathInZooKeeper Path of state handle to remove (expected to start with a '/') + * @param callback The callback after the operation finishes + * @throws Exception If the ZooKeeper operation fails + */ + public void remove(String pathInZooKeeper, BackgroundCallback callback) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + checkNotNull(callback, "Background callback"); + + client.delete().deletingChildrenIfNeeded().inBackground(callback).forPath(pathInZooKeeper); + } + + /** + * Discards a state handle and removes it from ZooKeeper. + * + *

If you only want to remove the state handle in ZooKeeper call {@link #remove(String)}. + * + * @param pathInZooKeeper Path of state handle to discard (expected to start with a '/') + * @throws Exception If the ZooKeeper or state handle operation fails + */ + public void removeAndDiscardState(String pathInZooKeeper) throws Exception { + checkNotNull(pathInZooKeeper, "Path in ZooKeeper"); + + StateHandle stateHandle = get(pathInZooKeeper); + + // Delete the state handle from ZooKeeper first + client.delete().deletingChildrenIfNeeded().forPath(pathInZooKeeper); + + // Discard the state handle only after it has been successfully deleted from ZooKeeper. + // Otherwise we might enter an illegal state after failures (with a state handle in + // ZooKeeper, which has already been discarded). + stateHandle.discardState(); + } + + /** + * Discards all available state handles and removes them from ZooKeeper. + * + * @throws Exception If a ZooKeeper or state handle operation fails + */ + public void removeAndDiscardAllState() throws Exception { + final List, String>> allStateHandles = getAll(); + + ZKPaths.deleteChildren( + client.getZookeeperClient().getZooKeeper(), + ZKPaths.fixForNamespace(client.getNamespace(), "/"), + false); + + // Discard the state handles only after they have been successfully deleted from ZooKeeper. + for (Tuple2, String> stateHandleAndPath : allStateHandles) { + stateHandleAndPath.f0.discardState(); + } + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala index 75ad20fea9d25..67d7a067ce73e 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobInfo.scala @@ -19,7 +19,7 @@ package org.apache.flink.runtime.jobmanager import akka.actor.ActorRef - +import org.apache.flink.runtime.akka.ListeningBehaviour /** * Utility class to store job information on the [[JobManager]]. The JobInfo stores which actor @@ -27,11 +27,20 @@ import akka.actor.ActorRef * Additionally, it stores whether the job was started in the detached mode. Detached means that * the submitting actor does not wait for the job result once the job has terminated. * + * Important: This class is serializable, but needs to be deserialized in the context of an actor + * system in order to resolve the client [[ActorRef]]. It is possible to serialize the Akka URL + * manually, but it is cumbersome and complicates testing in certain scenarios, where you need to + * make sure to resolve the correct [[ActorRef]]s when submitting jobs (RepointableActorRef vs. + * RemoteActorRef). + * * @param client Actor which submitted the job * @param start Starting time */ -class JobInfo(val client: ActorRef, val start: Long, - val sessionTimeout: Long) { +class JobInfo( + val client: ActorRef, + val listeningBehaviour: ListeningBehaviour, + val start: Long, + val sessionTimeout: Long) extends Serializable { var sessionAlive = sessionTimeout > 0 @@ -49,12 +58,16 @@ class JobInfo(val client: ActorRef, val start: Long, } } + override def toString = s"JobInfo(client: $client ($listeningBehaviour), start: $start)" + def setLastActive() = lastActive = System.currentTimeMillis() } object JobInfo{ - def apply(client: ActorRef, start: Long, - sessionTimeout: Long) = - new JobInfo(client, start, sessionTimeout) + def apply( + client: ActorRef, + listeningBehaviour: ListeningBehaviour, + start: Long, + sessionTimeout: Long) = new JobInfo(client, listeningBehaviour, start, sessionTimeout) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index d1031a8462c7d..a92fdcbe0b6bc 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -19,32 +19,40 @@ package org.apache.flink.runtime.jobmanager import java.io.{File, IOException} -import java.lang.reflect.{InvocationTargetException, Constructor} +import java.lang.reflect.{Constructor, InvocationTargetException} import java.net.InetSocketAddress -import java.util.{UUID, Collections} +import java.util.UUID import akka.actor.Status.Failure -import akka.actor.{Props, Terminated, PoisonPill, ActorRef, ActorSystem} +import akka.actor._ import akka.pattern.ask - import grizzled.slf4j.Logger - import org.apache.flink.api.common.{ExecutionConfig, JobID} import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration} import org.apache.flink.core.io.InputSplitAssigner import org.apache.flink.runtime.accumulators.AccumulatorSnapshot +import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour} import org.apache.flink.runtime.blob.BlobServer +import org.apache.flink.runtime.checkpoint.{CheckpointRecoveryFactory, StandaloneCheckpointRecoveryFactory, ZooKeeperCheckpointRecoveryFactory} import org.apache.flink.runtime.client._ +import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex} +import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager} import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator +import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID} +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener +import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} import org.apache.flink.runtime.jobmanager.web.WebInfoServer -import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService} +import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService} import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.messages.ArchiveMessages.ArchiveExecutionGraph import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged +import org.apache.flink.runtime.messages.JobManagerMessages._ import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect} +import org.apache.flink.runtime.messages.RegistrationMessages._ +import org.apache.flink.runtime.messages.TaskManagerMessages.{Heartbeat, SendStackTrace} import org.apache.flink.runtime.messages.TaskMessages.{PartitionState, UpdateTaskExecutionState} -import org.apache.flink.runtime.messages.accumulators._ +import org.apache.flink.runtime.messages.accumulators.{AccumulatorMessage, AccumulatorResultStringsFound, AccumulatorResultsErroneous, AccumulatorResultsFound, RequestAccumulatorResults, RequestAccumulatorResultsStringified} import org.apache.flink.runtime.messages.checkpoint.{AbstractCheckpointMessage, AcknowledgeCheckpoint} import org.apache.flink.runtime.messages.webmonitor._ import org.apache.flink.runtime.process.ProcessReaper @@ -52,25 +60,16 @@ import org.apache.flink.runtime.security.SecurityUtils import org.apache.flink.runtime.security.SecurityUtils.FlinkSecuredRunner import org.apache.flink.runtime.taskmanager.TaskManager import org.apache.flink.runtime.util._ -import org.apache.flink.runtime.webmonitor.{WebMonitorUtils, WebMonitor} -import org.apache.flink.runtime.{FlinkActor, StreamingMode, LeaderSessionMessageFilter} -import org.apache.flink.runtime.LogMessages -import org.apache.flink.runtime.akka.{ListeningBehaviour, AkkaUtils} -import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager -import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager} -import org.apache.flink.runtime.jobgraph.{JobVertexID, JobGraph, JobStatus} -import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler} -import org.apache.flink.runtime.messages.JobManagerMessages._ -import org.apache.flink.runtime.messages.RegistrationMessages._ -import org.apache.flink.runtime.messages.TaskManagerMessages.{SendStackTrace, Heartbeat} -import org.apache.flink.util.{NetUtils, SerializedValue, ExceptionUtils, InstantiationUtil} +import org.apache.flink.runtime.webmonitor.{WebMonitor, WebMonitorUtils} +import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages, StreamingMode} +import org.apache.flink.util.{ExceptionUtils, InstantiationUtil, NetUtils} +import scala.collection.JavaConverters._ +import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool import scala.language.postfixOps -import scala.collection.JavaConverters._ -import scala.concurrent.ExecutionContext.Implicits.global /** @@ -111,17 +110,22 @@ class JobManager( protected val delayBetweenRetries: Long, protected val timeout: FiniteDuration, protected val mode: StreamingMode, - protected val leaderElectionService: LeaderElectionService) + protected val leaderElectionService: LeaderElectionService, + protected val submittedJobGraphs : SubmittedJobGraphStore, + protected val checkpointRecoveryFactory : CheckpointRecoveryFactory) extends FlinkActor with LeaderSessionMessageFilter // mixin oder is important, we want filtering after logging with LogMessages // mixin order is important, we want first logging - with LeaderContender { + with LeaderContender + with SubmittedJobGraphListener { override val log = Logger(getClass) /** Either running or not yet archived jobs (session hasn't been ended). */ protected val currentJobs = scala.collection.mutable.HashMap[JobID, (ExecutionGraph, JobInfo)]() + protected val recoveryMode = RecoveryMode.fromConfig(flinkConfiguration) + var leaderSessionID: Option[UUID] = None /** @@ -139,6 +143,22 @@ class JobManager( "start.", e) throw new RuntimeException("Could not start the leader election service.", e) } + + try { + submittedJobGraphs.start(this) + } catch { + case e: Exception => + log.error("Could not start the submitted job graphs service.", e) + throw new RuntimeException("Could not start the submitted job graphs service.", e) + } + + try { + checkpointRecoveryFactory.start() + } catch { + case e: Exception => + log.error("Could not start the checkpoint recovery service.", e) + throw new RuntimeException("Could not start the checkpoint recovery service.", e) + } } override def postStop(): Unit = { @@ -160,6 +180,18 @@ class JobManager( case e: Exception => log.error("Could not properly shutdown the leader election service.") } + try { + submittedJobGraphs.stop() + } catch { + case e: Exception => log.error("Could not properly stop the submitted job graphs service.") + } + + try { + checkpointRecoveryFactory.stop() + } catch { + case e: Exception => log.error("Could not properly stop the checkpoint recovery service.") + } + if (archive != ActorRef.noSender) { archive ! decorateMessage(PoisonPill) } @@ -192,12 +224,21 @@ class JobManager( // confirming the leader session ID might be blocking, thus do it in a future future{ leaderElectionService.confirmLeaderSessionID(newLeaderSessionID.orNull) + + // TODO (critical next step) This needs to be more flexible and robust (e.g. wait for task + // managers etc.) + if (recoveryMode != RecoveryMode.STANDALONE) { + context.system.scheduler.scheduleOnce(new FiniteDuration(delayBetweenRetries, + MILLISECONDS), self, decorateMessage(RecoverAllJobs))(context.dispatcher) + } }(context.dispatcher) case RevokeLeadership => log.info(s"JobManager ${self.path.toSerializationFormat} was revoked leadership.") - cancelAndClearEverything(new Exception("JobManager is no longer the leader.")) + future { + cancelAndClearEverything(new Exception("JobManager is no longer the leader.")) + }(context.dispatcher) // disconnect the registered task managers instanceManager.getAllRegisteredInstances.asScala.foreach { @@ -270,7 +311,62 @@ class JobManager( sender ! decorateMessage(instanceManager.getTotalNumberOfSlots) case SubmitJob(jobGraph, listeningBehaviour) => - submitJob(jobGraph, listeningBehaviour) + val client = sender() + + val jobInfo = new JobInfo(client, listeningBehaviour, System.currentTimeMillis(), + jobGraph.getSessionTimeout) + + future { + submitJob(jobGraph, jobInfo) + }(context.dispatcher) + + case RecoverJob(jobId) => + future { + // The ActorRef, which is part of the submitted job graph can only be deserialized in the + // scope of an actor system. + akka.serialization.JavaSerializer.currentSystem.withValue( + context.system.asInstanceOf[ExtendedActorSystem]) { + + log.info(s"Attempting to recover job $jobId.") + + val jobGraph = submittedJobGraphs.recoverJobGraph(jobId) + + if (jobGraph.isDefined) { + if (!leaderElectionService.hasLeadership()) { + // we've lost leadership. mission: abort. + log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.") + } + else { + recoverJobGraph(jobGraph.get) + } + } + else { + log.warn(s"Failed to recover job graph ${jobId}.") + } + } + }(context.dispatcher) + + case RecoverAllJobs => + future { + // The ActorRef, which is part of the submitted job graph can only be deserialized in the + // scope of an actor system. + akka.serialization.JavaSerializer.currentSystem.withValue( + context.system.asInstanceOf[ExtendedActorSystem]) { + + val jobGraphs = submittedJobGraphs.recoverJobGraphs().asScala + + if (!leaderElectionService.hasLeadership()) { + // we've lost leadership. mission: abort. + log.warn(s"Lost leadership during recovery. Aborting recovery of ${jobGraphs.size} " + + s"jobs.") + } + else { + log.debug(s"Attempting to recover ${jobGraphs.size} job graphs.") + + jobGraphs.foreach(recoverJobGraph(_)) + } + } + }(context.dispatcher) case CancelJob(jobID) => log.info(s"Trying to cancel job with ID $jobID.") @@ -378,10 +474,27 @@ class JobManager( if (newJobStatus.isTerminalState()) { jobInfo.end = timeStamp - // is the client waiting for the job result? - if (jobInfo.client != ActorRef.noSender) { - newJobStatus match { - case JobStatus.FINISHED => + future { + // TODO If removing the JobGraph from the SubmittedJobGraphsStore fails, the job will + // linger around and potentially be recovered at a later time. There is nothing we + // can do about that, but it should be communicated with the Client. + if (jobInfo.sessionAlive) { + jobInfo.setLastActive() + val lastActivity = jobInfo.lastActive + context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) { + // remove only if no activity occurred in the meantime + if (lastActivity == jobInfo.lastActive) { + removeJob(jobID) + } + } + } else { + removeJob(jobID) + } + + // is the client waiting for the job result? + if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) { + newJobStatus match { + case JobStatus.FINISHED => try { val accumulatorResults = executionGraph.getAccumulatorsSerialized() val result = new SerializedJobExecutionResult( @@ -399,47 +512,37 @@ class JobManager( jobInfo.client ! decorateMessage(JobResultFailure( new SerializedThrowable(exception))) } - case JobStatus.CANCELED => - // the error may be packed as a serialized throwable - val unpackedError = SerializedThrowable.get( - error, executionGraph.getUserClassLoader()) - - jobInfo.client ! decorateMessage(JobResultFailure( - new SerializedThrowable( - new JobCancellationException(jobID, "Job was cancelled.", unpackedError)))) - - case JobStatus.FAILED => - val unpackedError = SerializedThrowable.get( - error, executionGraph.getUserClassLoader()) - - jobInfo.client ! decorateMessage(JobResultFailure( - new SerializedThrowable( - new JobExecutionException(jobID, "Job execution failed.", unpackedError)))) - - case x => - val exception = new JobExecutionException(jobID, s"$x is not a terminal state.") - jobInfo.client ! decorateMessage(JobResultFailure( - new SerializedThrowable(exception))) - throw exception - } - } - if (jobInfo.sessionAlive) { - jobInfo.setLastActive() - val lastActivity = jobInfo.lastActive - context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) { - // remove only if no activity occurred in the meantime - if (lastActivity == jobInfo.lastActive) { - removeJob(jobID) + case JobStatus.CANCELED => + // the error may be packed as a serialized throwable + val unpackedError = SerializedThrowable.get( + error, executionGraph.getUserClassLoader()) + + jobInfo.client ! decorateMessage(JobResultFailure( + new SerializedThrowable( + new JobCancellationException(jobID, "Job was cancelled.", unpackedError)))) + + case JobStatus.FAILED => + val unpackedError = SerializedThrowable.get( + error, executionGraph.getUserClassLoader()) + + jobInfo.client ! decorateMessage(JobResultFailure( + new SerializedThrowable( + new JobExecutionException(jobID, "Job execution failed.", unpackedError)))) + + case x => + val exception = new JobExecutionException(jobID, s"$x is not a terminal state.") + jobInfo.client ! decorateMessage(JobResultFailure( + new SerializedThrowable(exception))) + throw exception } } - } else { - removeJob(jobID) - } - + }(context.dispatcher) } case None => - removeJob(jobID) + future { + removeJob(jobID) + }(context.dispatcher) } case ScheduleOrUpdateConsumers(jobId, partitionId) => @@ -601,11 +704,12 @@ class JobManager( * graph and the execution vertices are queued for scheduling. * * @param jobGraph representing the Flink job - * @param listeningBehaviour specifies the listening behaviour of the sender. + * @param jobInfo the job info + * @param isRecovery Flag indicating whether this is a recovery or initial submission */ - private def submitJob(jobGraph: JobGraph, listeningBehaviour: ListeningBehaviour): Unit = { + private def submitJob(jobGraph: JobGraph, jobInfo: JobInfo, isRecovery: Boolean = false): Unit = { if (jobGraph == null) { - sender() ! decorateMessage(JobResultFailure( + jobInfo.client ! decorateMessage(JobResultFailure( new SerializedThrowable( new JobSubmissionException(null, "JobGraph must not be null.") ) @@ -616,7 +720,7 @@ class JobManager( val jobName = jobGraph.getName var executionGraph: ExecutionGraph = null - log.info(s"Received job ${jobId} (${jobName}).") + log.info(s"Submitting job $jobId ($jobName)" + (if (isRecovery) " (Recovery)" else "") + ".") try { // Important: We need to make sure that the library registration is the first action, @@ -629,7 +733,7 @@ class JobManager( catch { case t: Throwable => throw new JobSubmissionException(jobId, - "Cannot set up the user code libraries: " + t.getMessage, t) + "Cannot set up the user code libraries: " + t.getMessage, t) } val userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID) @@ -642,18 +746,10 @@ class JobManager( throw new JobSubmissionException(jobId, "The given job is empty") } - val client = if(listeningBehaviour == ListeningBehaviour.DETACHED) { - // The client does not want to receive the SerializedJobExecutionResult - ActorRef.noSender - } else { - // Send the job execution result back to the sender - sender - } - // see if there already exists an ExecutionGraph for the corresponding job ID executionGraph = currentJobs.get(jobGraph.getJobID) match { - case Some((graph, jobInfo)) => - jobInfo.setLastActive() + case Some((graph, currentJobInfo)) => + currentJobInfo.setLastActive() graph case None => val graph = new ExecutionGraph( @@ -665,11 +761,7 @@ class JobManager( jobGraph.getUserJarBlobKeys, jobGraph.getClasspaths, userCodeLoader) - val jobInfo = JobInfo( - client, - System.currentTimeMillis(), - jobGraph.getSessionTimeout) - currentJobs.put(jobGraph.getJobID, (graph, jobInfo)) + graph } @@ -683,7 +775,7 @@ class JobManager( executionGraph.setDelayBeforeRetrying(delayBetweenRetries) executionGraph.setScheduleMode(jobGraph.getScheduleMode()) executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling()) - + try { executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph)) } @@ -692,7 +784,7 @@ class JobManager( log.warn("Cannot create JSON plan for job", t) executionGraph.setJsonPlan("{}") } - + // initialize the vertices that have a master initialization hook // file output formats create directories here, input formats create splits if (log.isDebugEnabled) { @@ -702,62 +794,67 @@ class JobManager( val numSlots = scheduler.getTotalNumberOfSlots() for (vertex <- jobGraph.getVertices.asScala) { - val executableClass = vertex.getInvokableClassName if (executableClass == null || executableClass.length == 0) { throw new JobSubmissionException(jobId, s"The vertex ${vertex.getID} (${vertex.getName}) has no invokable class.") } - if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) { - vertex.setParallelism(numSlots) - } + if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) { + vertex.setParallelism(numSlots) + } - try { - vertex.initializeOnMaster(userCodeLoader) - } - catch { + try { + vertex.initializeOnMaster(userCodeLoader) + } + catch { case t: Throwable => throw new JobExecutionException(jobId, "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage, t) - } - } + } + } - // topologically sort the job vertices and attach the graph to the existing one - val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources() - if (log.isDebugEnabled) { - log.debug(s"Adding ${sortedTopology.size()} vertices from " + - s"job graph ${jobId} (${jobName}).") - } - executionGraph.attachJobGraph(sortedTopology) + // topologically sort the job vertices and attach the graph to the existing one + val sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources() + if (log.isDebugEnabled) { + log.debug(s"Adding ${sortedTopology.size()} vertices from " + + s"job graph ${jobId} (${jobName}).") + } + executionGraph.attachJobGraph(sortedTopology) - if (log.isDebugEnabled) { - log.debug("Successfully created execution graph from job " + - s"graph ${jobId} (${jobName}).") - } + if (log.isDebugEnabled) { + log.debug("Successfully created execution graph from job " + + s"graph ${jobId} (${jobName}).") + } - // configure the state checkpointing - val snapshotSettings = jobGraph.getSnapshotSettings - if (snapshotSettings != null) { + // configure the state checkpointing + val snapshotSettings = jobGraph.getSnapshotSettings + if (snapshotSettings != null) { + val jobId = jobGraph.getJobID() - val idToVertex: JobVertexID => ExecutionJobVertex = id => { - val vertex = executionGraph.getJobVertex(id) - if (vertex == null) { - throw new JobSubmissionException(jobId, - "The snapshot checkpointing settings refer to non-existent vertex " + id) - } - vertex - } + val idToVertex: JobVertexID => ExecutionJobVertex = id => { + val vertex = executionGraph.getJobVertex(id) + if (vertex == null) { + throw new JobSubmissionException(jobId, + "The snapshot checkpointing settings refer to non-existent vertex " + id) + } + vertex + } - val triggerVertices: java.util.List[ExecutionJobVertex] = + val triggerVertices: java.util.List[ExecutionJobVertex] = snapshotSettings.getVerticesToTrigger().asScala.map(idToVertex).asJava - val ackVertices: java.util.List[ExecutionJobVertex] = + val ackVertices: java.util.List[ExecutionJobVertex] = snapshotSettings.getVerticesToAcknowledge().asScala.map(idToVertex).asJava - val confirmVertices: java.util.List[ExecutionJobVertex] = + val confirmVertices: java.util.List[ExecutionJobVertex] = snapshotSettings.getVerticesToConfirm().asScala.map(idToVertex).asJava + val completedCheckpoints = checkpointRecoveryFactory + .createCompletedCheckpoints(jobId, userCodeLoader) + + val checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(jobId) + executionGraph.enableSnapshotCheckpointing( snapshotSettings.getCheckpointInterval, snapshotSettings.getCheckpointTimeout, @@ -765,23 +862,39 @@ class JobManager( ackVertices, confirmVertices, context.system, - leaderSessionID.orNull) + leaderSessionID.orNull, + checkpointIdCounter, + completedCheckpoints, + recoveryMode) } // get notified about job status changes executionGraph.registerJobStatusListener( new AkkaActorGateway(self, leaderSessionID.orNull)) - if (listeningBehaviour == ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES) { + if (jobInfo.listeningBehaviour == ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES) { // the sender wants to be notified about state changes - val gateway = new AkkaActorGateway(sender(), leaderSessionID.orNull) + val gateway = new AkkaActorGateway(jobInfo.client, leaderSessionID.orNull) executionGraph.registerExecutionListener(gateway) executionGraph.registerJobStatusListener(gateway) } + if (isRecovery) { + executionGraph.restoreLatestCheckpointedState() + } + else { + submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo)) + } + + // Add the job graph only after everything is finished. Otherwise there can be races in + // tests, which check the currentJobs (for example before killing a JM). + if (!currentJobs.contains(jobId)) { + currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo)) + } + // done with submitting the job - sender() ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID)) + jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID)) } catch { case t: Throwable => @@ -800,33 +913,61 @@ class JobManager( new JobExecutionException(jobId, s"Failed to submit job ${jobId} (${jobName})", t) } - sender() ! decorateMessage(JobResultFailure(new SerializedThrowable(rt))) + jobInfo.client ! decorateMessage(JobResultFailure(new SerializedThrowable(rt))) return } - // NOTE: Scheduling the job for execution is a separate action from the job submission. - // The success of submitting the job must be independent from the success of scheduling - // the job. - try { - log.info(s"Scheduling job ${executionGraph.getJobName}.") - executionGraph.scheduleForExecution(scheduler) - } - catch { - case t: Throwable => try { - executionGraph.fail(t) + if (leaderElectionService.hasLeadership) { + // There is a small chance that multiple job managers schedule the same job after if they + // try to recover at the same time. This will eventually be noticed, but can not be ruled + // out from the beginning. + + // NOTE: Scheduling the job for execution is a separate action from the job submission. + // The success of submitting the job must be independent from the success of scheduling + // the job. + try { + log.info(s"Scheduling job $jobId ($jobName).") + + executionGraph.scheduleForExecution(scheduler) } catch { - case tt: Throwable => { - log.error("Error while marking ExecutionGraph as failed.", tt) + case t: Throwable => try { + executionGraph.fail(t) + } + catch { + case tt: Throwable => { + log.error("Error while marking ExecutionGraph as failed.", tt) + } } } } + else { + // Remove the job graph. Otherwise it will be lingering around and possibly removed from + // ZooKeeper by this JM. + currentJobs.remove(jobId) + + log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " + + "this. I am not scheduling the job for execution.") + } + } + } + + /** + * Submits the job if it is not already one of our current jobs. + * + * @param jobGraph Job to recover + */ + private def recoverJobGraph(jobGraph: SubmittedJobGraph): Unit = { + if (!currentJobs.contains(jobGraph.getJobId)) { + future { + submitJob(jobGraph.getJobGraph(), jobGraph.getJobInfo(), isRecovery = true) + }(context.dispatcher) } } /** * Dedicated handler for checkpoint messages. - * + * * @param actorMessage The checkpoint actor message. */ private def handleCheckpointMessage(actorMessage: AbstractCheckpointMessage): Unit = { @@ -837,13 +978,15 @@ class JobManager( case Some((graph, _)) => val coordinator = graph.getCheckpointCoordinator() if (coordinator != null) { - try { - coordinator.receiveAcknowledgeMessage(ackMessage) - } - catch { - case t: Throwable => - log.error(s"Error in CheckpointCoordinator while processing $ackMessage", t) - } + future { + try { + coordinator.receiveAcknowledgeMessage(ackMessage) + } + catch { + case t: Throwable => + log.error(s"Error in CheckpointCoordinator while processing $ackMessage", t) + } + }(context.dispatcher) } else { log.error( @@ -1021,30 +1164,46 @@ class JobManager( } /** - * Removes the job and sends it to the MemoryArchivist + * Removes the job and sends it to the MemoryArchivist. + * + * This should be called asynchronously. Removing the job from the [[SubmittedJobGraphStore]] + * might block. Therefore be careful not to block the actor thread. + * * @param jobID ID of the job to remove and archive */ private def removeJob(jobID: JobID): Unit = { currentJobs.synchronized { - currentJobs.remove(jobID) match { + // Don't remove the job yet... + currentJobs.get(jobID) match { case Some((eg, _)) => + try { + // ...otherwise, we can have lingering resources when there is a concurrent shutdown + // and the ZooKeeper client is closed. Not removing the job immediately allow the + // shutdown to release all resources. + submittedJobGraphs.removeJobGraph(jobID) + } catch { + case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t) + } + try { eg.prepareForArchiving() + archive ! decorateMessage(ArchiveExecutionGraph(jobID, eg)) } catch { case t: Throwable => log.error(s"Could not prepare the execution graph $eg for " + "archiving.", t) } + currentJobs.remove(jobID) case None => } + } - try { - libraryCacheManager.unregisterJob(jobID) - } catch { - case t: Throwable => - log.error(s"Could not properly unregister job $jobID form the library cache.", t) - } + try { + libraryCacheManager.unregisterJob(jobID) + } catch { + case t: Throwable => + log.error(s"Could not properly unregister job $jobID form the library cache.", t) } } @@ -1054,17 +1213,21 @@ class JobManager( * @param cause Cause for the cancelling. */ private def cancelAndClearEverything(cause: Throwable) { - for((jobID, (eg, jobInfo)) <- currentJobs) { + for ((jobID, (eg, jobInfo)) <- currentJobs) { + try { + submittedJobGraphs.removeJobGraph(jobID) + } + catch { + case t: Throwable => { + log.error("Error during submitted job graph clean up.", t) + } + } + eg.fail(cause) - if(jobInfo.client != ActorRef.noSender) { + if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) { jobInfo.client ! decorateMessage( - Failure( - new JobExecutionException( - jobID, - "All jobs are cancelled and cleared.", - cause) - )) + Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause))) } } @@ -1080,6 +1243,25 @@ class JobManager( self ! decorateMessage(RevokeLeadership) } + override def onAddedJobGraph(jobId: JobID): Unit = { + if (leaderSessionID.isDefined && !currentJobs.contains(jobId)) { + self ! decorateMessage(RecoverJob(jobId)) + } + } + + override def onRemovedJobGraph(jobId: JobID): Unit = { + if (leaderSessionID.isDefined) { + currentJobs.get(jobId).foreach( + job => + future { + // Fail the execution graph + job._1.fail(new IllegalStateException("Another JobManager removed the job from " + + "ZooKeeper.")) + }(context.dispatcher) + ) + } + } + override def getAddress: String = { AkkaUtils.getAkkaURL(context.system, self) } @@ -1167,7 +1349,7 @@ object JobManager { System.exit(STARTUP_FAILURE_RETURN_CODE) } - if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) { + if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) { // address and will not be reachable from anyone remote if (listeningPort != 0) { val message = "Config parameter '" + ConfigConstants.JOB_MANAGER_IPC_PORT_KEY + @@ -1228,7 +1410,7 @@ object JobManager { * * @param configuration The configuration object for the JobManager. * @param executionMode The execution mode in which to run. Execution mode LOCAL will spawn an - * additional TaskManager in the same process. + * an additional TaskManager in the same process. * @param streamingMode The streaming mode to run the system in (streaming vs. batch-only) * @param listeningAddress The hostname where the JobManager should listen for messages. * @param listeningPort The port where the JobManager should listen for messages. @@ -1489,7 +1671,7 @@ object JobManager { // high availability mode val port: Int = - if (ZooKeeperUtils.isZooKeeperHighAvailabilityEnabled(configuration)) { + if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) { LOG.info("Starting JobManager in High-Availability Mode") configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0) @@ -1533,7 +1715,9 @@ object JobManager { Long, // delay between retries FiniteDuration, // timeout Int, // number of archived jobs - LeaderElectionService) = { + LeaderElectionService, + SubmittedJobGraphStore, + CheckpointRecoveryFactory) = { val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration) @@ -1597,10 +1781,31 @@ object JobManager { } } - val leaderElectionService = leaderElectionServiceOption match { - case Some(les) => les - case None => LeaderElectionUtils.createLeaderElectionService(configuration) - } + // Create recovery related components + val (leaderElectionService, submittedJobGraphs, checkpointRecoveryFactory) = + RecoveryMode.fromConfig(configuration) match { + case RecoveryMode.STANDALONE => + val leaderElectionService = leaderElectionServiceOption match { + case Some(les) => les + case None => new StandaloneLeaderElectionService() + } + + (leaderElectionService, + new StandaloneSubmittedJobGraphStore(), + new StandaloneCheckpointRecoveryFactory()) + + case RecoveryMode.ZOOKEEPER => + val client = ZooKeeperUtils.startCuratorFramework(configuration) + + val leaderElectionService = leaderElectionServiceOption match { + case Some(les) => les + case None => ZooKeeperUtils.createLeaderElectionService(client, configuration) + } + + (leaderElectionService, + ZooKeeperUtils.createSubmittedJobGraphs(client, configuration), + new ZooKeeperCheckpointRecoveryFactory(client, configuration)) + } (executionContext, instanceManager, @@ -1608,9 +1813,11 @@ object JobManager { libraryCacheManager, executionRetries, delayBetweenRetries, - timeout, - archiveCount, - leaderElectionService) + timeout, + archiveCount, + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) } /** @@ -1642,6 +1849,7 @@ object JobManager { jobManagerClass, archiveClass) } + /** * Starts the JobManager and job archiver based on the given configuration, in the * given actor system. @@ -1655,28 +1863,30 @@ object JobManager { * @param streamingMode The mode to run the system in (streaming vs. batch-only) * @param jobManagerClass The class of the JobManager to be started * @param archiveClass The class of the MemoryArchivist to be started - * + * * @return A tuple of references (JobManager Ref, Archiver Ref) */ def startJobManagerActors( - configuration: Configuration, - actorSystem: ActorSystem, - jobMangerActorName: Option[String], - archiveActorName: Option[String], - streamingMode: StreamingMode, - jobManagerClass: Class[_ <: JobManager], - archiveClass: Class[_ <: MemoryArchivist]) - : (ActorRef, ActorRef) = { + configuration: Configuration, + actorSystem: ActorSystem, + jobMangerActorName: Option[String], + archiveActorName: Option[String], + streamingMode: StreamingMode, + jobManagerClass: Class[_ <: JobManager], + archiveClass: Class[_ <: MemoryArchivist]) + : (ActorRef, ActorRef) = { val (executionContext, - instanceManager, - scheduler, - libraryCacheManager, - executionRetries, - delayBetweenRetries, - timeout, - archiveCount, - leaderElectionService) = createJobManagerComponents( + instanceManager, + scheduler, + libraryCacheManager, + executionRetries, + delayBetweenRetries, + timeout, + archiveCount, + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) = createJobManagerComponents( configuration, None) @@ -1700,7 +1910,9 @@ object JobManager { delayBetweenRetries, timeout, streamingMode, - leaderElectionService) + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) val jobManager: ActorRef = jobMangerActorName match { case Some(actorName) => actorSystem.actorOf(jobManagerProps, actorName) diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala index c29df885ccc45..d77662242e198 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala @@ -65,6 +65,18 @@ object JobManagerMessages { listeningBehaviour: ListeningBehaviour) extends RequiresLeaderSessionID + /** + * Triggers the recovery of the job with the given ID. + * + * @param jobId ID of the job to recover + */ + case class RecoverJob(jobId: JobID) extends RequiresLeaderSessionID + + /** + * Triggers recovery of all available jobs. + */ + case class RecoverAllJobs() extends RequiresLeaderSessionID + /** * Cancels a job with the given [[jobID]] at the JobManager. The result of the cancellation is * sent back to the sender as a [[CancellationResponse]] message. @@ -354,6 +366,10 @@ object JobManagerMessages { // -------------------------------------------------------------------------- // Utility methods to allow simpler case object access from Java // -------------------------------------------------------------------------- + + def getRequestJobStatus(jobId : JobID) : AnyRef = { + RequestJobStatus(jobId) + } def getRequestNumberRegisteredTaskManager : AnyRef = { RequestNumberRegisteredTaskManager diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 77e977fc3abb8..b795ecdb6efda 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -95,9 +95,7 @@ abstract class FlinkMiniCluster( implicit val timeout = AkkaUtils.getTimeout(userConfiguration) - val recoveryMode = RecoveryMode.valueOf(configuration.getString( - ConfigConstants.RECOVERY_MODE, - ConfigConstants.DEFAULT_RECOVERY_MODE).toUpperCase) + val recoveryMode = RecoveryMode.fromConfig(configuration) val numJobManagers = getNumberOfJobManagers diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 1330b66bdc382..f6ee5c57758ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -18,14 +18,13 @@ package org.apache.flink.runtime.checkpoint; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.Execution; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; @@ -34,13 +33,25 @@ import java.util.Iterator; import java.util.List; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + /** * Tests for the checkpoint coordinator. */ public class CheckpointCoordinatorTest { - + private static final ClassLoader cl = Thread.currentThread().getContextClassLoader(); - + @Test public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { try { @@ -50,7 +61,7 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { // create some mock Execution vertices that receive the checkpoint trigger messages ExecutionVertex triggerVertex1 = mock(ExecutionVertex.class); ExecutionVertex triggerVertex2 = mock(ExecutionVertex.class); - + // create some mock Execution vertices that need to ack the checkpoint final ExecutionAttemptID ackAttemptID1 = new ExecutionAttemptID(); final ExecutionAttemptID ackAttemptID2 = new ExecutionAttemptID(); @@ -59,10 +70,12 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 1, 600000, + jid, 600000, new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, new ExecutionVertex[] { ackVertex1, ackVertex2 }, - new ExecutionVertex[] {}, cl ); + new ExecutionVertex[] {}, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -103,10 +116,12 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() { // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 1, 600000, + jid, 600000, new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, new ExecutionVertex[] { ackVertex1, ackVertex2 }, - new ExecutionVertex[] {}, cl ); + new ExecutionVertex[] {}, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -138,17 +153,18 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { final ExecutionAttemptID triggerAttemptID2 = new ExecutionAttemptID(); ExecutionVertex triggerVertex1 = mockExecutionVertex(triggerAttemptID1); ExecutionVertex triggerVertex2 = mockExecutionVertex(triggerAttemptID2); - + // create some mock Execution vertices that receive the checkpoint trigger messages ExecutionVertex ackVertex1 = mock(ExecutionVertex.class); ExecutionVertex ackVertex2 = mock(ExecutionVertex.class); // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 1, 600000, + jid, 600000, new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, new ExecutionVertex[] { ackVertex1, ackVertex2 }, - new ExecutionVertex[] {}, cl ); + new ExecutionVertex[] {}, cl, new StandaloneCheckpointIDCounter(), new + StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -168,13 +184,13 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { fail(e.getMessage()); } } - + @Test public void testTriggerAndConfirmSimpleCheckpoint() { try { final JobID jid = new JobID(); final long timestamp = System.currentTimeMillis(); - + // create some mock Execution vertices that receive the checkpoint trigger messages final ExecutionAttemptID attemptID1 = new ExecutionAttemptID(); final ExecutionAttemptID attemptID2 = new ExecutionAttemptID(); @@ -183,24 +199,26 @@ public void testTriggerAndConfirmSimpleCheckpoint() { // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 1, 600000, + jid, 600000, new ExecutionVertex[] { vertex1, vertex2 }, new ExecutionVertex[] { vertex1, vertex2 }, - new ExecutionVertex[] { vertex1, vertex2 }, cl); - + new ExecutionVertex[] { vertex1, vertex2 }, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); + assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - + // trigger the first checkpoint. this should succeed assertTrue(coord.triggerCheckpoint(timestamp)); - + // validate that we have a pending checkpoint assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - + long checkpointId = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); PendingCheckpoint checkpoint = coord.getPendingCheckpoints().get(checkpointId); - + assertNotNull(checkpoint); assertEquals(checkpointId, checkpoint.getCheckpointId()); assertEquals(timestamp, checkpoint.getCheckpointTimestamp()); @@ -210,7 +228,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertEquals(0, checkpoint.getCollectedStates().size()); assertFalse(checkpoint.isDiscarded()); assertFalse(checkpoint.isFullyAcknowledged()); - + // check that the vertices received the trigger checkpoint message { TriggerCheckpoint expectedMessage1 = new TriggerCheckpoint(jid, attemptID1, checkpointId, timestamp); @@ -218,7 +236,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { verify(vertex1, times(1)).sendMessageToCurrentExecution(eq(expectedMessage1), eq(attemptID1)); verify(vertex2, times(1)).sendMessageToCurrentExecution(eq(expectedMessage2), eq(attemptID2)); } - + // acknowledge from one of the tasks coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointId)); assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); @@ -233,15 +251,15 @@ public void testTriggerAndConfirmSimpleCheckpoint() { // acknowledge the other task. coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId)); - + // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed assertTrue(checkpoint.isDiscarded()); - + // the now we should have a completed checkpoint assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); - + // validate that the relevant tasks got a confirmation message { NotifyCheckpointComplete confirmMessage1 = new NotifyCheckpointComplete(jid, attemptID1, checkpointId, timestamp); @@ -249,13 +267,13 @@ public void testTriggerAndConfirmSimpleCheckpoint() { verify(vertex1, times(1)).sendMessageToCurrentExecution(eq(confirmMessage1), eq(attemptID1)); verify(vertex2, times(1)).sendMessageToCurrentExecution(eq(confirmMessage2), eq(attemptID2)); } - - SuccessfulCheckpoint success = coord.getSuccessfulCheckpoints().get(0); + + CompletedCheckpoint success = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, success.getJobId()); assertEquals(timestamp, success.getTimestamp()); assertEquals(checkpoint.getCheckpointId(), success.getCheckpointID()); assertTrue(success.getStates().isEmpty()); - + // --------------- // trigger another checkpoint and see that this one replaces the other checkpoint // --------------- @@ -265,11 +283,11 @@ public void testTriggerAndConfirmSimpleCheckpoint() { long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew)); - + assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); - - SuccessfulCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0); + + CompletedCheckpoint successNew = coord.getSuccessfulCheckpoints().get(0); assertEquals(jid, successNew.getJobId()); assertEquals(timestampNew, successNew.getTimestamp()); assertEquals(checkpointIdNew, successNew.getCheckpointID()); @@ -295,8 +313,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { fail(e.getMessage()); } } - - + @Test public void testMultipleConcurrentCheckpoints() { try { @@ -305,7 +322,7 @@ public void testMultipleConcurrentCheckpoints() { final long timestamp2 = timestamp1 + 8617; // create some mock execution vertices - + final ExecutionAttemptID triggerAttemptID1 = new ExecutionAttemptID(); final ExecutionAttemptID triggerAttemptID2 = new ExecutionAttemptID(); @@ -314,23 +331,25 @@ public void testMultipleConcurrentCheckpoints() { final ExecutionAttemptID ackAttemptID3 = new ExecutionAttemptID(); final ExecutionAttemptID commitAttemptID = new ExecutionAttemptID(); - + ExecutionVertex triggerVertex1 = mockExecutionVertex(triggerAttemptID1); ExecutionVertex triggerVertex2 = mockExecutionVertex(triggerAttemptID2); ExecutionVertex ackVertex1 = mockExecutionVertex(ackAttemptID1); ExecutionVertex ackVertex2 = mockExecutionVertex(ackAttemptID2); ExecutionVertex ackVertex3 = mockExecutionVertex(ackAttemptID3); - + ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID); - + // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 2, 600000, + jid, 600000, new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 }, - new ExecutionVertex[] { commitVertex }, cl); - + new ExecutionVertex[] { commitVertex }, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE); + assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -339,7 +358,7 @@ public void testMultipleConcurrentCheckpoints() { assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); - + PendingCheckpoint pending1 = coord.getPendingCheckpoints().values().iterator().next(); long checkpointId1 = pending1.getCheckpointId(); @@ -348,10 +367,10 @@ public void testMultipleConcurrentCheckpoints() { new TriggerCheckpoint(jid, triggerAttemptID1, checkpointId1, timestamp1), triggerAttemptID1); verify(triggerVertex2, times(1)).sendMessageToCurrentExecution( new TriggerCheckpoint(jid, triggerAttemptID2, checkpointId1, timestamp1), triggerAttemptID2); - + // acknowledge one of the three tasks coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1)); - + // start the second checkpoint // trigger the first checkpoint. this should succeed assertTrue(coord.triggerCheckpoint(timestamp2)); @@ -373,23 +392,23 @@ public void testMultipleConcurrentCheckpoints() { new TriggerCheckpoint(jid, triggerAttemptID1, checkpointId2, timestamp2), triggerAttemptID1); verify(triggerVertex2, times(1)).sendMessageToCurrentExecution( new TriggerCheckpoint(jid, triggerAttemptID2, checkpointId2, timestamp2), triggerAttemptID2); - + // we acknowledge the remaining two tasks from the first // checkpoint and two tasks from the second checkpoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2)); - + // now, the first checkpoint should be confirmed assertEquals(1, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); assertTrue(pending1.isDiscarded()); - + // the first confirm message should be out verify(commitVertex, times(1)).sendMessageToCurrentExecution( new NotifyCheckpointComplete(jid, commitAttemptID, checkpointId1, timestamp1), commitAttemptID); - + // send the last remaining ack for the second checkpoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2)); @@ -401,17 +420,17 @@ public void testMultipleConcurrentCheckpoints() { // the second commit message should be out verify(commitVertex, times(1)).sendMessageToCurrentExecution( new NotifyCheckpointComplete(jid, commitAttemptID, checkpointId2, timestamp2), commitAttemptID); - + // validate the committed checkpoints - List scs = coord.getSuccessfulCheckpoints(); - - SuccessfulCheckpoint sc1 = scs.get(0); + List scs = coord.getSuccessfulCheckpoints(); + + CompletedCheckpoint sc1 = scs.get(0); assertEquals(checkpointId1, sc1.getCheckpointID()); assertEquals(timestamp1, sc1.getTimestamp()); assertEquals(jid, sc1.getJobId()); assertTrue(sc1.getStates().isEmpty()); - - SuccessfulCheckpoint sc2 = scs.get(1); + + CompletedCheckpoint sc2 = scs.get(1); assertEquals(checkpointId2, sc2.getCheckpointID()); assertEquals(timestamp2, sc2.getTimestamp()); assertEquals(jid, sc2.getJobId()); @@ -453,10 +472,12 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 10, 600000, + jid, 600000, new ExecutionVertex[] { triggerVertex1, triggerVertex2 }, new ExecutionVertex[] { ackVertex1, ackVertex2, ackVertex3 }, - new ExecutionVertex[] { commitVertex }, cl); + new ExecutionVertex[] { commitVertex }, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(10, cl), RecoveryMode.STANDALONE); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -513,13 +534,13 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // into a successful checkpoint assertTrue(pending1.isDiscarded()); assertTrue(pending2.isDiscarded()); - + assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); // validate the committed checkpoints - List scs = coord.getSuccessfulCheckpoints(); - SuccessfulCheckpoint success = scs.get(0); + List scs = coord.getSuccessfulCheckpoints(); + CompletedCheckpoint success = scs.get(0); assertEquals(checkpointId2, success.getCheckpointID()); assertEquals(timestamp2, success.getTimestamp()); assertEquals(jid, success.getJobId()); @@ -531,7 +552,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // send the last remaining ack for the first checkpoint. This should not do anything coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1)); - + coord.shutdown(); } catch (Exception e) { @@ -539,8 +560,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { fail(e.getMessage()); } } - - + @Test public void testCheckpointTimeoutIsolated() { try { @@ -565,22 +585,24 @@ public void testCheckpointTimeoutIsolated() { // set up the coordinator // the timeout for the checkpoint is a 200 milliseconds - + CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 2, 200, + jid, 200, new ExecutionVertex[] { triggerVertex }, new ExecutionVertex[] { ackVertex1, ackVertex2 }, - new ExecutionVertex[] { commitVertex }, cl); - + new ExecutionVertex[] { commitVertex }, cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE); + // trigger a checkpoint, partially acknowledged assertTrue(coord.triggerCheckpoint(timestamp)); assertEquals(1, coord.getNumberOfPendingCheckpoints()); - + PendingCheckpoint checkpoint = coord.getPendingCheckpoints().values().iterator().next(); assertFalse(checkpoint.isDiscarded()); - + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId())); - + // wait until the checkpoint must have expired. // we check every 250 msecs conservatively for 5 seconds // to give even slow build servers a very good chance of completing this @@ -591,7 +613,7 @@ public void testCheckpointTimeoutIsolated() { while (!checkpoint.isDiscarded() && coord.getNumberOfPendingCheckpoints() > 0 && System.currentTimeMillis() < deadline); - + assertTrue("Checkpoint was not canceled by the timeout", checkpoint.isDiscarded()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -599,7 +621,7 @@ public void testCheckpointTimeoutIsolated() { // no confirm message must have been sent verify(commitVertex, times(0)) .sendMessageToCurrentExecution(any(NotifyCheckpointComplete.class), any(ExecutionAttemptID.class)); - + coord.shutdown(); } catch (Exception e) { @@ -607,7 +629,7 @@ public void testCheckpointTimeoutIsolated() { fail(e.getMessage()); } } - + @Test public void handleMessagesForNonExistingCheckpoints() { try { @@ -625,27 +647,28 @@ public void handleMessagesForNonExistingCheckpoints() { ExecutionVertex ackVertex1 = mockExecutionVertex(ackAttemptID1); ExecutionVertex ackVertex2 = mockExecutionVertex(ackAttemptID2); ExecutionVertex commitVertex = mockExecutionVertex(commitAttemptID); - + CheckpointCoordinator coord = new CheckpointCoordinator( - jid, 2, 200000, + jid, 200000, new ExecutionVertex[] { triggerVertex }, new ExecutionVertex[] { ackVertex1, ackVertex2 }, - new ExecutionVertex[] { commitVertex }, cl); + new ExecutionVertex[] { commitVertex }, cl, new StandaloneCheckpointIDCounter + (), new StandaloneCompletedCheckpointStore(2, cl), RecoveryMode.STANDALONE); assertTrue(coord.triggerCheckpoint(timestamp)); - + long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next(); - + // send some messages that do not belong to either the job or the any // of the vertices that need to be acknowledged. // non of the messages should throw an exception - + // wrong job id coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(new JobID(), ackAttemptID1, checkpointId)); - + // unknown checkpoint coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, 1L)); - + // unknown ack vertex coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, new ExecutionAttemptID(), checkpointId)); @@ -660,15 +683,16 @@ public void handleMessagesForNonExistingCheckpoints() { private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID) { return mockExecutionVertex(attemptID, ExecutionState.RUNNING); } - + private static ExecutionVertex mockExecutionVertex(ExecutionAttemptID attemptID, ExecutionState state) { final Execution exec = mock(Execution.class); when(exec.getAttemptId()).thenReturn(attemptID); when(exec.getState()).thenReturn(state); ExecutionVertex vertex = mock(ExecutionVertex.class); + when(vertex.getJobvertexId()).thenReturn(new JobVertexID()); when(vertex.getCurrentExecutionAttempt()).thenReturn(exec); - + return vertex; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java new file mode 100644 index 0000000000000..96c4eea724a10 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTest.java @@ -0,0 +1,194 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.junit.Assert.assertEquals; + +public abstract class CheckpointIDCounterTest extends TestLogger { + + protected abstract CheckpointIDCounter createCompletedCheckpoints() throws Exception; + + public static class StandaloneCheckpointIDCounterTest extends CheckpointIDCounterTest { + + @Override + protected CheckpointIDCounter createCompletedCheckpoints() throws Exception { + return new StandaloneCheckpointIDCounter(); + } + } + + public static class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTest { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + } + + @Override + protected CheckpointIDCounter createCompletedCheckpoints() throws Exception { + return new ZooKeeperCheckpointIDCounter(ZooKeeper.getClient(), + "/checkpoint-id-counter"); + } + } + + // --------------------------------------------------------------------------------------------- + + /** + * Tests serial increment and get calls. + */ + @Test + public void testSerialIncrementAndGet() throws Exception { + final CheckpointIDCounter counter = createCompletedCheckpoints(); + + try { + counter.start(); + + assertEquals(1, counter.getAndIncrement()); + assertEquals(2, counter.getAndIncrement()); + assertEquals(3, counter.getAndIncrement()); + assertEquals(4, counter.getAndIncrement()); + } + finally { + counter.stop(); + } + } + + /** + * Tests concurrent increment and get calls from multiple Threads and verifies that the numbers + * counts strictly increasing. + */ + @Test + public void testConcurrentGetAndIncrement() throws Exception { + // Config + final int numThreads = 8; + + // Setup + final CountDownLatch startLatch = new CountDownLatch(1); + final CheckpointIDCounter counter = createCompletedCheckpoints(); + counter.start(); + + ExecutorService executor = null; + try { + executor = Executors.newFixedThreadPool(numThreads); + + List>> resultFutures = new ArrayList<>(numThreads); + + for (int i = 0; i < numThreads; i++) { + resultFutures.add(executor.submit(new Incrementer(startLatch, counter))); + } + + // Kick off the incrementing + startLatch.countDown(); + + final int expectedTotal = numThreads * Incrementer.NumIncrements; + + List all = new ArrayList<>(expectedTotal); + + // Get the counts + for (Future> result : resultFutures) { + List counts = result.get(); + + for (long val : counts) { + all.add(val); + } + } + + // Verify + Collections.sort(all); + + assertEquals(expectedTotal, all.size()); + + long current = 0; + for (long val : all) { + // Incrementing counts + assertEquals(++current, val); + } + + // The final count + assertEquals(expectedTotal + 1, counter.getAndIncrement()); + } + finally { + if (executor != null) { + executor.shutdown(); + } + + counter.stop(); + } + } + + /** + * Task repeatedly incrementing the {@link CheckpointIDCounter}. + */ + private static class Incrementer implements Callable> { + + /** Total number of {@link CheckpointIDCounter#getAndIncrement()} calls. */ + private final static int NumIncrements = 128; + + private final CountDownLatch startLatch; + + private final CheckpointIDCounter counter; + + public Incrementer(CountDownLatch startLatch, CheckpointIDCounter counter) { + this.startLatch = startLatch; + this.counter = counter; + } + + @Override + public List call() throws Exception { + final Random rand = new Random(); + final List counts = new ArrayList<>(); + + // Wait for the main thread to kick off execution + this.startLatch.await(); + + for (int i = 0; i < NumIncrements; i++) { + counts.add(counter.getAndIncrement()); + + // To get some "random" interleaving ;) + Thread.sleep(rand.nextInt(20)); + } + + return counts; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 08cb0a3b54d35..32c15bf613af1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.RecoveryMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.state.LocalStateHandle; import org.apache.flink.runtime.state.StateHandle; @@ -79,10 +80,12 @@ public void testSetState() { map.put(statelessId, stateless); - CheckpointCoordinator coord = new CheckpointCoordinator(jid, 1, 200000L, + CheckpointCoordinator coord = new CheckpointCoordinator(jid, 200000L, new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }, new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }, - new ExecutionVertex[0], cl); + new ExecutionVertex[0], cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); // create ourselves a checkpoint with state final long timestamp = 34623786L; @@ -148,10 +151,12 @@ public void testStateOnlyPartiallyAvailable() { map.put(statelessId, stateless); - CheckpointCoordinator coord = new CheckpointCoordinator(jid, 1, 200000L, + CheckpointCoordinator coord = new CheckpointCoordinator(jid, 200000L, new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }, new ExecutionVertex[] { stateful1, stateful2, stateful3, stateless1, stateless2 }, - new ExecutionVertex[0], cl); + new ExecutionVertex[0], cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); // create ourselves a checkpoint with state final long timestamp = 34623786L; @@ -188,10 +193,12 @@ public void testStateOnlyPartiallyAvailable() { @Test public void testNoCheckpointAvailable() { try { - CheckpointCoordinator coord = new CheckpointCoordinator(new JobID(), 1, 200000L, + CheckpointCoordinator coord = new CheckpointCoordinator(new JobID(), 200000L, new ExecutionVertex[] { mock(ExecutionVertex.class) }, new ExecutionVertex[] { mock(ExecutionVertex.class) }, - new ExecutionVertex[0], cl); + new ExecutionVertex[0], cl, + new StandaloneCheckpointIDCounter(), + new StandaloneCompletedCheckpointStore(1, cl), RecoveryMode.STANDALONE); try { coord.restoreLatestCheckpointedState(new HashMap(), true, false); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java new file mode 100644 index 0000000000000..9e3c605547d40 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java @@ -0,0 +1,297 @@ +/* + * 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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.CheckpointMessagesTest; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test for basic {@link CompletedCheckpointStore} contract. + */ +public abstract class CompletedCheckpointStoreTest extends TestLogger { + + /** + * Creates the {@link CompletedCheckpointStore} implementation to be tested. + */ + protected abstract CompletedCheckpointStore createCompletedCheckpoints( + int maxNumberOfCheckpointsToRetain, ClassLoader userLoader) throws Exception; + + // --------------------------------------------------------------------------------------------- + + // Verify that discarded checkpoints are called with the correct class loader + private final ClassLoader userClassLoader = ClassLoader.getSystemClassLoader(); + + /** + * Tests that at least one checkpoint needs to be retained. + */ + @Test(expected = Exception.class) + public void testExceptionOnNoRetainedCheckpoints() throws Exception { + createCompletedCheckpoints(0, userClassLoader); + } + + /** + * Tests adding and getting a checkpoint. + */ + @Test + public void testAddAndGetLatestCheckpoint() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader); + + // Empty state + assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints()); + assertEquals(0, checkpoints.getAllCheckpoints().size()); + + TestCheckpoint[] expected = new TestCheckpoint[] { + createCheckpoint(0), createCheckpoint(1) }; + + // Add and get latest + checkpoints.addCheckpoint(expected[0]); + assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints()); + verifyCheckpoint(expected[0], checkpoints.getLatestCheckpoint()); + + checkpoints.addCheckpoint(expected[1]); + assertEquals(2, checkpoints.getNumberOfRetainedCheckpoints()); + verifyCheckpoint(expected[1], checkpoints.getLatestCheckpoint()); + } + + /** + * Tests that adding more checkpoints than retained discards the correct checkpoints (using + * the correct class loader). + */ + @Test + public void testAddCheckpointMoreThanMaxRetained() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(1, userClassLoader); + + TestCheckpoint[] expected = new TestCheckpoint[] { + createCheckpoint(0), createCheckpoint(1), + createCheckpoint(2), createCheckpoint(3) + }; + + // Add checkpoints + checkpoints.addCheckpoint(expected[0]); + assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints()); + + for (int i = 1; i < expected.length; i++) { + checkpoints.addCheckpoint(expected[i]); + + // The ZooKeeper implementation discards asynchronously + expected[i - 1].awaitDiscard(); + assertTrue(expected[i - 1].isDiscarded()); + assertEquals(userClassLoader, expected[i - 1].getDiscardClassLoader()); + + assertEquals(1, checkpoints.getNumberOfRetainedCheckpoints()); + } + } + + /** + * Tests that + *

    + *
  • {@link CompletedCheckpointStore#getLatestCheckpoint()} returns null,
  • + *
  • {@link CompletedCheckpointStore#getAllCheckpoints()} returns an empty list,
  • + *
  • {@link CompletedCheckpointStore#getNumberOfRetainedCheckpoints()} returns 0.
  • + *
+ */ + @Test + public void testEmptyState() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(1, userClassLoader); + + assertNull(checkpoints.getLatestCheckpoint()); + assertEquals(0, checkpoints.getAllCheckpoints().size()); + assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints()); + } + + /** + * Tests that all added checkpoints are returned. + */ + @Test + public void testGetAllCheckpoints() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader); + + TestCheckpoint[] expected = new TestCheckpoint[] { + createCheckpoint(0), createCheckpoint(1), + createCheckpoint(2), createCheckpoint(3) + }; + + for (TestCheckpoint checkpoint : expected) { + checkpoints.addCheckpoint(checkpoint); + } + + List actual = checkpoints.getAllCheckpoints(); + + assertEquals(expected.length, actual.size()); + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], actual.get(i)); + } + } + + /** + * Tests that all checkpoints are discarded (using the correct class loader). + */ + @Test + public void testDiscardAllCheckpoints() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(4, userClassLoader); + + TestCheckpoint[] expected = new TestCheckpoint[] { + createCheckpoint(0), createCheckpoint(1), + createCheckpoint(2), createCheckpoint(3) + }; + + for (TestCheckpoint checkpoint : expected) { + checkpoints.addCheckpoint(checkpoint); + } + + checkpoints.discardAllCheckpoints(); + + // Empty state + assertNull(checkpoints.getLatestCheckpoint()); + assertEquals(0, checkpoints.getAllCheckpoints().size()); + assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints()); + + // All have been discarded + for (TestCheckpoint checkpoint : expected) { + // The ZooKeeper implementation discards asynchronously + checkpoint.awaitDiscard(); + assertTrue(checkpoint.isDiscarded()); + assertEquals(userClassLoader, checkpoint.getDiscardClassLoader()); + } + } + + // --------------------------------------------------------------------------------------------- + + protected TestCheckpoint createCheckpoint(int id) throws IOException { + return createCheckpoint(id, 4); + } + + protected TestCheckpoint createCheckpoint(int id, int numberOfStates) + throws IOException { + + JobVertexID jvid = new JobVertexID(); + + ArrayList taskStates = new ArrayList<>(); + + for (int i = 0; i < numberOfStates; i++) { + SerializedValue> stateHandle = new SerializedValue>( + new CheckpointMessagesTest.MyHandle()); + + taskStates.add(new StateForTask(stateHandle, jvid, i)); + } + + return new TestCheckpoint(new JobID(), id, 0, taskStates); + } + + private void verifyCheckpoint(CompletedCheckpoint expected, CompletedCheckpoint actual) { + assertEquals(expected.getJobId(), actual.getJobId()); + assertEquals(expected.getCheckpointID(), actual.getCheckpointID()); + assertEquals(expected.getTimestamp(), actual.getTimestamp()); + + List expectedStates = expected.getStates(); + List actualStates = actual.getStates(); + + assertEquals(expectedStates.size(), actualStates.size()); + + for (int i = 0; i < expectedStates.size(); i++) { + assertEquals(expectedStates.get(i), actualStates.get(i)); + } + } + + /** + * A test {@link CompletedCheckpoint}. We want to verify that the correct class loader is + * used when discarding. Spying on a regular {@link CompletedCheckpoint} instance with + * Mockito doesn't work, because it it breaks serializability. + */ + protected static class TestCheckpoint extends CompletedCheckpoint { + + private static final long serialVersionUID = 4211419809665983026L; + + private boolean isDiscarded; + + // Latch for test variants which discard asynchronously + private transient final CountDownLatch discardLatch = new CountDownLatch(1); + + private transient ClassLoader discardClassLoader; + + public TestCheckpoint( + JobID jobId, + long checkpointId, + long timestamp, + ArrayList states) { + + super(jobId, checkpointId, timestamp, states); + } + + @Override + public void discard(ClassLoader userClassLoader) { + super.discard(userClassLoader); + + if (!isDiscarded) { + this.discardClassLoader = userClassLoader; + this.isDiscarded = true; + + if (discardLatch != null) { + discardLatch.countDown(); + } + } + } + + public boolean isDiscarded() { + return isDiscarded; + } + + public void awaitDiscard() throws InterruptedException { + if (discardLatch != null) { + discardLatch.await(); + } + } + + public ClassLoader getDiscardClassLoader() { + return discardClassLoader; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TestCheckpoint that = (TestCheckpoint) o; + + return getJobId().equals(that.getJobId()) + && getCheckpointID() == that.getCheckpointID(); + } + + @Override + public int hashCode() { + return getJobId().hashCode() + (int) getCheckpointID(); + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java new file mode 100644 index 0000000000000..beccbf89d3015 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStoreTest.java @@ -0,0 +1,33 @@ +/* + * 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.checkpoint; + +/** + * Tests for basic {@link CompletedCheckpointStore} contract. + */ +public class StandaloneCompletedCheckpointStoreTest extends CompletedCheckpointStoreTest { + + @Override + protected CompletedCheckpointStore createCompletedCheckpoints( + int maxNumberOfCheckpointsToRetain, + ClassLoader userClassLoader) throws Exception { + + return new StandaloneCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userClassLoader); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java new file mode 100644 index 0000000000000..4c6ddfd6ab496 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -0,0 +1,101 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.state.LocalStateHandle; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for basic {@link CompletedCheckpointStore} contract and ZooKeeper state handling. + */ +public class ZooKeeperCompletedCheckpointStoreITCase extends CompletedCheckpointStoreTest { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static String CheckpointsPath = "/checkpoints"; + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + } + + @Override + protected CompletedCheckpointStore createCompletedCheckpoints(int maxNumberOfCheckpointsToRetain, + ClassLoader userLoader) throws Exception { + + return new ZooKeeperCompletedCheckpointStore(maxNumberOfCheckpointsToRetain, userLoader, + ZooKeeper.createClient(), CheckpointsPath, new LocalStateHandle + .LocalStateHandleProvider()); + } + + // --------------------------------------------------------------------------------------------- + + /** + * Tests that older checkpoints are cleaned up at startup. + */ + @Test + public void testRecover() throws Exception { + CompletedCheckpointStore checkpoints = createCompletedCheckpoints(3, ClassLoader + .getSystemClassLoader()); + + TestCheckpoint[] expected = new TestCheckpoint[] { + createCheckpoint(0), createCheckpoint(1), createCheckpoint(2) + }; + + // Add multiple checkpoints + checkpoints.addCheckpoint(expected[0]); + checkpoints.addCheckpoint(expected[1]); + checkpoints.addCheckpoint(expected[2]); + + // All three should be in ZK + assertEquals(3, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size()); + + // Recover + checkpoints.recover(); + + // Only the latest one should be in ZK + Deadline deadline = new FiniteDuration(1, TimeUnit.MINUTES).fromNow(); + + // Retry this operation, because removal is asynchronous + while (deadline.hasTimeLeft() && ZooKeeper.getClient() + .getChildren().forPath(CheckpointsPath).size() != 1) { + + Thread.sleep(Math.min(100, deadline.timeLeft().toMillis())); + } + + assertEquals(1, ZooKeeper.getClient().getChildren().forPath(CheckpointsPath).size()); + assertEquals(expected[2], checkpoints.getLatestCheckpoint()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java index 5a5ef5747a8ae..fa61acf09fe78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java @@ -178,6 +178,10 @@ public void testRegisterAndDownload() { // un-register them again libCache.unregisterTask(jid, executionId); + + // Don't fail if called again + libCache.unregisterTask(jid, executionId); + assertEquals(0, libCache.getNumberOfReferenceHolders(jid)); // library is still cached (but not associated with job any more) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java index 56e5bdebb91f8..ca8810b4eb9f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactoryTest.java @@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.ConnectionID; -import org.apache.flink.runtime.net.NetUtils; +import org.apache.flink.util.NetUtils; import org.junit.Ignore; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java new file mode 100644 index 0000000000000..ac250bd82a1e9 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java @@ -0,0 +1,460 @@ +/* + * 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.jobmanager; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Props; +import akka.actor.UntypedActor; +import akka.testkit.TestActorRef; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.leaderelection.TestingListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.messages.JobManagerMessages.JobStatusResponse; +import org.apache.flink.runtime.messages.JobManagerMessages.LeaderSessionMessage; +import org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.apache.flink.runtime.testingUtils.TestingCluster; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; +import org.apache.flink.runtime.testutils.JobManagerProcess; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.util.TestLogger; +import org.apache.zookeeper.data.Stat; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Queue; +import java.util.UUID; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Tests recovery of {@link SubmittedJobGraph} instances. + */ +public class JobManagerSubmittedJobGraphsRecoveryITCase extends TestLogger { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES); + + private static final File FileStateBackendBasePath; + + static { + try { + FileStateBackendBasePath = CommonTestUtils.createTempDirectory(); + } + catch (IOException e) { + throw new RuntimeException("Error in test setup. Could not create directory.", e); + } + } + + @AfterClass + public static void tearDown() throws Exception { + ZooKeeper.shutdown(); + + if (FileStateBackendBasePath != null) { + FileUtils.deleteDirectory(FileStateBackendBasePath); + } + } + + @Before + public void cleanUp() throws Exception { + if (FileStateBackendBasePath != null) { + FileUtils.cleanDirectory(FileStateBackendBasePath); + } + + ZooKeeper.deleteAll(); + } + + // --------------------------------------------------------------------------------------------- + + /** + * Tests that the recovery state is cleaned up after a JobManager stops. + */ + @Test + public void testJobManagerCleanUp() throws Exception { + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); + + // Configure the cluster + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 1); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + + TestingCluster flink = new TestingCluster(config, false, false, StreamingMode.STREAMING); + + try { + final Deadline deadline = TestTimeOut.fromNow(); + + // Start the JobManager and TaskManager + flink.start(true); + + JobGraph jobGraph = createBlockingJobGraph(); + + ActorGateway jobManager = flink.getLeaderGateway(deadline.timeLeft()); + + // Submit the job + jobManager.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); + + // Wait for the job to start + JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING, + jobManager, deadline.timeLeft()); + } + finally { + flink.shutdown(); + } + + // Verify that everything is clean + verifyCleanRecoveryState(config); + } + + /** + * Tests that submissions to non-leaders are handled. + */ + @Test + public void testSubmitJobToNonLeader() throws Exception { + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); + + // Configure the cluster + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 2); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + + TestingCluster flink = new TestingCluster(config, false, false, StreamingMode.STREAMING); + + try { + final Deadline deadline = TestTimeOut.fromNow(); + + // Start the JobManager and TaskManager + flink.start(true); + + JobGraph jobGraph = createBlockingJobGraph(); + + List bothJobManagers = flink.getJobManagersAsJava(); + + ActorGateway leadingJobManager = flink.getLeaderGateway(deadline.timeLeft()); + + ActorGateway nonLeadingJobManager; + if (bothJobManagers.get(0).equals(leadingJobManager.actor())) { + nonLeadingJobManager = new AkkaActorGateway(bothJobManagers.get(1), null); + } + else { + nonLeadingJobManager = new AkkaActorGateway(bothJobManagers.get(0), null); + } + + // Submit the job + nonLeadingJobManager.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); + + // Wait for the job to start. We are asking the *leading** JM here although we've + // submitted the job to the non-leading JM. This is the behaviour under test. + JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING, + leadingJobManager, deadline.timeLeft()); + + // Make sure that the **non-leading** JM has actually removed the job graph from her + // local state. + boolean success = false; + while (!success && deadline.hasTimeLeft()) { + JobStatusResponse jobStatusResponse = JobManagerActorTestUtils.requestJobStatus( + jobGraph.getJobID(), nonLeadingJobManager, deadline.timeLeft()); + + if (jobStatusResponse instanceof JobManagerMessages.JobNotFound) { + success = true; + } + else { + Thread.sleep(100); + } + } + + if (!success) { + fail("Non-leading JM was still holding reference to the job graph."); + } + } + finally { + flink.shutdown(); + } + + // Verify that everything is clean + verifyCleanRecoveryState(config); + } + + /** + * Tests that clients receive updates after recovery by a new leader. + */ + @Test + public void testClientNonDetachedListeningBehaviour() throws Exception { + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); + + // Test actor system + ActorSystem testSystem = null; + + // JobManager setup. Start the job managers as separate processes in order to not run the + // actors postStop, which cleans up all running jobs. + JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2]; + + LeaderRetrievalService leaderRetrievalService = null; + + ActorSystem taskManagerSystem = null; + + try { + final Deadline deadline = TestTimeOut.fromNow(); + + // Test actor system + testSystem = AkkaUtils.createActorSystem(new Configuration(), + new Some<>(new Tuple2("localhost", 0))); + + // The job managers + jobManagerProcess[0] = new JobManagerProcess(0, config); + jobManagerProcess[1] = new JobManagerProcess(1, config); + + jobManagerProcess[0].createAndStart(); + jobManagerProcess[1].createAndStart(); + + // Leader listener + TestingListener leaderListener = new TestingListener(); + leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config); + leaderRetrievalService.start(leaderListener); + + // The task manager + taskManagerSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); + TaskManager.startTaskManagerComponentsAndActor( + config, taskManagerSystem, "localhost", + Option.empty(), Option.empty(), + false, StreamingMode.STREAMING, TaskManager.class); + + // Client test actor + TestActorRef clientRef = TestActorRef.create( + testSystem, Props.create(RecordingTestClient.class)); + + JobGraph jobGraph = createBlockingJobGraph(); + + { + // Initial submission + leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + + String leaderAddress = leaderListener.getAddress(); + UUID leaderId = leaderListener.getLeaderSessionID(); + + // The client + AkkaActorGateway client = new AkkaActorGateway(clientRef, leaderId); + + // Get the leader ref + ActorRef leaderRef = AkkaUtils.getActorRef( + leaderAddress, testSystem, deadline.timeLeft()); + ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId); + + // Submit the job in non-detached mode + leader.tell(new SubmitJob(jobGraph, + ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES), client); + + JobManagerActorTestUtils.waitForJobStatus( + jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft()); + } + + // Who's the boss? + JobManagerProcess leadingJobManagerProcess; + if (jobManagerProcess[0].getJobManagerAkkaURL().equals(leaderListener.getAddress())) { + leadingJobManagerProcess = jobManagerProcess[0]; + } + else { + leadingJobManagerProcess = jobManagerProcess[1]; + } + + // Kill the leading job manager process + leadingJobManagerProcess.destroy(); + + { + // Recovery by the standby JobManager + leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + + String leaderAddress = leaderListener.getAddress(); + UUID leaderId = leaderListener.getLeaderSessionID(); + + ActorRef leaderRef = AkkaUtils.getActorRef( + leaderAddress, testSystem, deadline.timeLeft()); + ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId); + + JobManagerActorTestUtils.waitForJobStatus( + jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft()); + + // Cancel the job + leader.tell(new JobManagerMessages.CancelJob(jobGraph.getJobID())); + } + + // Wait for the execution result + clientRef.underlyingActor().awaitJobResult(deadline.timeLeft().toMillis()); + + int jobSubmitSuccessMessages = 0; + for (Object msg : clientRef.underlyingActor().getMessages()) { + if (msg instanceof JobManagerMessages.JobSubmitSuccess) { + jobSubmitSuccessMessages++; + } + } + + // At least two submissions should be ack-ed (initial and recovery). This is quite + // conservative, but it is still possible that these messages are overtaken by the + // final message. + assertEquals(2, jobSubmitSuccessMessages); + } + catch (Throwable t) { + // In case of an error, print the job manager process logs. + if (jobManagerProcess[0] != null) { + jobManagerProcess[0].printProcessLog(); + } + + if (jobManagerProcess[1] != null) { + jobManagerProcess[1].printProcessLog(); + } + + t.printStackTrace(); + } + finally { + if (jobManagerProcess[0] != null) { + jobManagerProcess[0].destroy(); + } + + if (jobManagerProcess[1] != null) { + jobManagerProcess[1].destroy(); + } + + if (leaderRetrievalService != null) { + leaderRetrievalService.stop(); + } + + if (taskManagerSystem != null) { + taskManagerSystem.shutdown(); + } + + if (testSystem != null) { + testSystem.shutdown(); + } + } + } + + /** + * Simple recording client. + */ + private static class RecordingTestClient extends UntypedActor { + + private final Queue messages = new ConcurrentLinkedQueue<>(); + + private CountDownLatch jobResultLatch = new CountDownLatch(1); + + @Override + public void onReceive(Object message) throws Exception { + if (message instanceof LeaderSessionMessage) { + message = ((LeaderSessionMessage) message).message(); + } + + messages.add(message); + + // Check for job result + if (message instanceof JobManagerMessages.JobResultFailure || + message instanceof JobManagerMessages.JobResultSuccess) { + + jobResultLatch.countDown(); + } + } + + public Queue getMessages() { + return messages; + } + + public void awaitJobResult(long timeout) throws InterruptedException { + jobResultLatch.await(timeout, TimeUnit.MILLISECONDS); + } + } + + // --------------------------------------------------------------------------------------------- + + /** + * Creates a simple blocking JobGraph. + */ + private static JobGraph createBlockingJobGraph() { + JobGraph jobGraph = new JobGraph("Blocking program"); + + JobVertex jobVertex = new JobVertex("Blocking Vertex"); + jobVertex.setInvokableClass(Tasks.BlockingNoOpInvokable.class); + + jobGraph.addVertex(jobVertex); + + return jobGraph; + } + + /** + * Fails the test if the recovery state (file state backend and ZooKeeper) is not clean. + */ + private static void verifyCleanRecoveryState(Configuration config) throws Exception { + // File state backend empty + Collection stateHandles = FileUtils.listFiles( + FileStateBackendBasePath, TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE); + + if (!stateHandles.isEmpty()) { + fail("File state backend is not clean: " + stateHandles); + } + + // ZooKeeper + String currentJobsPath = config.getString( + ConfigConstants.ZOOKEEPER_JOBGRAPHS_PATH, + ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); + + Stat stat = ZooKeeper.getClient().checkExists().forPath(currentJobsPath); + + if (stat.getCversion() == 0) { + // Sanity check: verify that some changes have been performed + fail("ZooKeeper state for '" + currentJobsPath + "' has not been modified during " + + "this test. What are you testing?"); + } + + if (stat.getNumChildren() != 0) { + // Is everything clean again? + fail("ZooKeeper path '" + currentJobsPath + "' is not clean: " + + ZooKeeper.getClient().getChildren().forPath(currentJobsPath)); + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java new file mode 100644 index 0000000000000..753e7be2a0080 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneSubmittedJobGraphStoreTest.java @@ -0,0 +1,53 @@ +/* + * 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.jobmanager; + +import akka.actor.ActorRef; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StandaloneSubmittedJobGraphStoreTest { + + /** + * Tests that all operations work and don't change the state. + */ + @Test + public void testNoOps() throws Exception { + StandaloneSubmittedJobGraphStore jobGraphs = new StandaloneSubmittedJobGraphStore(); + + SubmittedJobGraph jobGraph = new SubmittedJobGraph( + new JobGraph("testNoOps"), + new JobInfo(ActorRef.noSender(), ListeningBehaviour.DETACHED, 0, Integer.MAX_VALUE)); + + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + jobGraphs.putJobGraph(jobGraph); + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + jobGraphs.removeJobGraph(jobGraph.getJobGraph().getJobID()); + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + assertTrue(jobGraphs.recoverJobGraph(new JobID()).isEmpty()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java new file mode 100644 index 0000000000000..861a7134f4a24 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java @@ -0,0 +1,283 @@ +/* + * 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.jobmanager; + +import akka.actor.ActorRef; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener; +import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.CountDownLatch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests for basic {@link SubmittedJobGraphStore} contract. + */ +public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static LocalStateHandleProvider StateHandleProvider = + new LocalStateHandleProvider<>(); + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + } + + @Test + public void testPutAndRemoveJobGraph() throws Exception { + ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testPutAndRemoveJobGraph", + StateHandleProvider); + + try { + SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class); + + jobGraphs.start(listener); + + SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0); + + // Empty state + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + // Add initial + jobGraphs.putJobGraph(jobGraph); + + // Verify initial job graph + List actual = jobGraphs.recoverJobGraphs(); + assertEquals(1, actual.size()); + + verifyJobGraphs(jobGraph, actual.get(0)); + + // Update (same ID) + jobGraph = createSubmittedJobGraph(jobGraph.getJobId(), 1); + jobGraphs.putJobGraph(jobGraph); + + // Verify updated + actual = jobGraphs.recoverJobGraphs(); + assertEquals(1, actual.size()); + + verifyJobGraphs(jobGraph, actual.get(0)); + + // Remove + jobGraphs.removeJobGraph(jobGraph.getJobId()); + + // Empty state + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + // Nothing should have been notified + verify(listener, atMost(1)).onAddedJobGraph(any(JobID.class)); + verify(listener, never()).onRemovedJobGraph(any(JobID.class)); + + // Don't fail if called again + jobGraphs.removeJobGraph(jobGraph.getJobId()); + } + finally { + jobGraphs.stop(); + } + } + + @Test + public void testRecoverJobGraphs() throws Exception { + ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testRecoverJobGraphs", StateHandleProvider); + + try { + SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class); + + jobGraphs.start(listener); + + HashMap expected = new HashMap<>(); + JobID[] jobIds = new JobID[] { new JobID(), new JobID(), new JobID() }; + + expected.put(jobIds[0], createSubmittedJobGraph(jobIds[0], 0)); + expected.put(jobIds[1], createSubmittedJobGraph(jobIds[1], 1)); + expected.put(jobIds[2], createSubmittedJobGraph(jobIds[2], 2)); + + // Add all + for (SubmittedJobGraph jobGraph : expected.values()) { + jobGraphs.putJobGraph(jobGraph); + } + + List actual = jobGraphs.recoverJobGraphs(); + + assertEquals(expected.size(), actual.size()); + + for (SubmittedJobGraph jobGraph : actual) { + assertTrue(expected.containsKey(jobGraph.getJobId())); + + verifyJobGraphs(expected.get(jobGraph.getJobId()), jobGraph); + + jobGraphs.removeJobGraph(jobGraph.getJobId()); + } + + // Empty state + assertEquals(0, jobGraphs.recoverJobGraphs().size()); + + // Nothing should have been notified + verify(listener, atMost(expected.size())).onAddedJobGraph(any(JobID.class)); + verify(listener, never()).onRemovedJobGraph(any(JobID.class)); + } + finally { + jobGraphs.stop(); + } + } + + @Test + public void testConcurrentAddJobGraph() throws Exception { + ZooKeeperSubmittedJobGraphStore jobGraphs = null; + ZooKeeperSubmittedJobGraphStore otherJobGraphs = null; + + try { + jobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider); + + otherJobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testConcurrentAddJobGraph", StateHandleProvider); + + + SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0); + SubmittedJobGraph otherJobGraph = createSubmittedJobGraph(new JobID(), 0); + + SubmittedJobGraphListener listener = mock(SubmittedJobGraphListener.class); + + final JobID[] actualOtherJobId = new JobID[1]; + final CountDownLatch sync = new CountDownLatch(1); + + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + actualOtherJobId[0] = (JobID) invocation.getArguments()[0]; + sync.countDown(); + + return null; + } + }).when(listener).onAddedJobGraph(any(JobID.class)); + + // Test + jobGraphs.start(listener); + otherJobGraphs.start(null); + + jobGraphs.putJobGraph(jobGraph); + + // Everything is cool... not much happening ;) + verify(listener, never()).onAddedJobGraph(any(JobID.class)); + verify(listener, never()).onRemovedJobGraph(any(JobID.class)); + + // This bad boy adds the other job graph + otherJobGraphs.putJobGraph(otherJobGraph); + + // Wait for the cache to call back + sync.await(); + + verify(listener, times(1)).onAddedJobGraph(any(JobID.class)); + verify(listener, never()).onRemovedJobGraph(any(JobID.class)); + + assertEquals(otherJobGraph.getJobId(), actualOtherJobId[0]); + } + finally { + if (jobGraphs != null) { + jobGraphs.stop(); + } + + if (otherJobGraphs != null) { + otherJobGraphs.stop(); + } + } + } + + @Test(expected = IllegalStateException.class) + public void testUpdateJobGraphYouDidNotGetOrAdd() throws Exception { + ZooKeeperSubmittedJobGraphStore jobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider); + + ZooKeeperSubmittedJobGraphStore otherJobGraphs = new ZooKeeperSubmittedJobGraphStore( + ZooKeeper.createClient(), "/testUpdateJobGraphYouDidNotGetOrAdd", StateHandleProvider); + + jobGraphs.start(null); + otherJobGraphs.start(null); + + SubmittedJobGraph jobGraph = createSubmittedJobGraph(new JobID(), 0); + + jobGraphs.putJobGraph(jobGraph); + + otherJobGraphs.putJobGraph(jobGraph); + } + + // --------------------------------------------------------------------------------------------- + + private SubmittedJobGraph createSubmittedJobGraph(JobID jobId, long start) { + final JobGraph jobGraph = new JobGraph(jobId, "Test JobGraph"); + + final JobVertex jobVertex = new JobVertex("Test JobVertex"); + jobVertex.setParallelism(1); + + jobGraph.addVertex(jobVertex); + + final JobInfo jobInfo = new JobInfo( + ActorRef.noSender(), ListeningBehaviour.DETACHED, start, Integer.MAX_VALUE); + + return new SubmittedJobGraph(jobGraph, jobInfo); + } + + protected void verifyJobGraphs(SubmittedJobGraph expected, SubmittedJobGraph actual) + throws Exception { + + JobGraph expectedJobGraph = expected.getJobGraph(); + JobGraph actualJobGraph = actual.getJobGraph(); + + assertEquals(expectedJobGraph.getName(), actualJobGraph.getName()); + assertEquals(expectedJobGraph.getJobID(), actualJobGraph.getJobID()); + + JobInfo expectedJobInfo = expected.getJobInfo(); + JobInfo actualJobInfo = actual.getJobInfo(); + + assertEquals(expectedJobInfo.listeningBehaviour(), actualJobInfo.listeningBehaviour()); + assertEquals(expectedJobInfo.start(), actualJobInfo.start()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java index 753bbab63f0b1..bbd8fadcd3ad2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java @@ -25,19 +25,25 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import akka.util.Timeout; +import org.apache.curator.framework.CuratorFramework; import org.apache.curator.test.TestingServer; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.StreamingMode; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.instance.InstanceManager; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.testingUtils.TestingJobManager; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.util.LeaderElectionUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -151,8 +157,19 @@ public void testLeaderReelection() throws Exception { } private Props createJobManagerProps(Configuration configuration) throws Exception { - LeaderElectionService leaderElectionService = LeaderElectionUtils. - createLeaderElectionService(configuration); + LeaderElectionService leaderElectionService; + if (RecoveryMode.fromConfig(configuration) == RecoveryMode.STANDALONE) { + leaderElectionService = new StandaloneLeaderElectionService(); + } + else { + CuratorFramework client = ZooKeeperUtils.startCuratorFramework(configuration); + leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client, + configuration); + } + + // We don't need recovery in this test + SubmittedJobGraphStore submittedJobGraphStore = new StandaloneSubmittedJobGraphStore(); + CheckpointRecoveryFactory checkpointRecoveryFactory = new StandaloneCheckpointRecoveryFactory(); return Props.create( TestingJobManager.class, @@ -166,7 +183,9 @@ private Props createJobManagerProps(Configuration configuration) throws Exceptio 1L, AkkaUtils.getDefaultTimeout(), StreamingMode.BATCH_ONLY, - leaderElectionService + leaderElectionService, + submittedJobGraphStore, + checkpointRecoveryFactory ); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java index c4fccd7671b87..ea058f4bc9836 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java @@ -27,6 +27,8 @@ */ public class TestingLeaderElectionService implements LeaderElectionService, Serializable { + private static final long serialVersionUID = -8007939683948014574L; + private LeaderContender contender; private boolean hasLeadership = false; @@ -51,10 +53,12 @@ public boolean hasLeadership() { } public void isLeader(UUID leaderSessionID) { + hasLeadership = true; contender.grantLeadership(leaderSessionID); } public void notLeader() { + hasLeadership = false; contender.revokeLeadership(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java index bb60415f002aa..aae1840e9ef12 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.leaderelection; +import org.apache.curator.framework.CuratorFramework; import org.apache.curator.test.TestingServer; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.util.LeaderElectionUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; @@ -41,7 +42,7 @@ import java.net.UnknownHostException; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; public class ZooKeeperLeaderRetrievalTest extends TestLogger{ @@ -92,7 +93,11 @@ public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exc Thread thread; + CuratorFramework[] client = new CuratorFramework[2]; + try { + client[0] = ZooKeeperUtils.startCuratorFramework(config); + client[1] = ZooKeeperUtils.startCuratorFramework(config); InetSocketAddress wrongInetSocketAddress = new InetSocketAddress(InetAddress.getByName("1.1.1.1"), 1234); @@ -116,7 +121,7 @@ public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exc String correctAddress = JobManager.getRemoteJobManagerAkkaURL(correctInetSocketAddress, Option.empty()); - faultyLeaderElectionService = LeaderElectionUtils.createLeaderElectionService(config); + faultyLeaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[0], config); TestingContender wrongLeaderAddressContender = new TestingContender(wrongAddress, faultyLeaderElectionService); faultyLeaderElectionService.start(wrongLeaderAddressContender); @@ -127,7 +132,7 @@ public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exc thread.start(); - leaderElectionService = LeaderElectionUtils.createLeaderElectionService(config); + leaderElectionService = ZooKeeperUtils.createLeaderElectionService(client[1], config); TestingContender correctLeaderAddressContender = new TestingContender(correctAddress, leaderElectionService); Thread.sleep(sleepingTime); @@ -155,6 +160,14 @@ public void testConnectingAddressRetrievalWithDelayedLeaderElection() throws Exc if (leaderElectionService != null) { leaderElectionService.stop(); } + + if (client[0] != null) { + client[0].close(); + } + + if (client[1] != null) { + client[1].close(); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java index 68575e5b09a25..087e0fd8d7ebd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java @@ -78,7 +78,7 @@ private static void testSerializabilityEqualsHashCode(Serializable o) throws IOE assertNotNull(copy.toString()); } - private static class MyHandle implements StateHandle { + public static class MyHandle implements StateHandle { private static final long serialVersionUID = 8128146204128728332L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java index 837b64364d79d..cd40c82539fa4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelTest.java @@ -26,10 +26,10 @@ import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate; -import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.JobManager; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; @@ -38,6 +38,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages.JobNotFound; import org.apache.flink.runtime.testingUtils.TestingCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; import org.apache.flink.types.IntValue; import org.junit.Test; import scala.concurrent.Await; @@ -48,7 +49,6 @@ import static org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; import static org.apache.flink.runtime.messages.JobManagerMessages.CancellationFailure; import static org.apache.flink.runtime.messages.JobManagerMessages.RequestJobStatus; -import static org.junit.Assert.fail; public class TaskCancelTest { @@ -109,30 +109,21 @@ public void testCancelUnion() throws Exception { // Wait for the job to make some progress and then cancel awaitRunning( - flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), - jobGraph.getJobID(), - TestingUtils.TESTING_DURATION()); + flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), + jobGraph.getJobID(), + TestingUtils.TESTING_DURATION()); Thread.sleep(5000); cancelJob( - flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), - jobGraph.getJobID(), - TestingUtils.TESTING_DURATION()); + flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), + jobGraph.getJobID(), + TestingUtils.TESTING_DURATION()); // Wait for the job to be cancelled - JobStatus status = awaitTermination( - flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), - jobGraph.getJobID(), - TestingUtils.TESTING_DURATION()); - - if (status == JobStatus.CANCELED) { - // Expected :-) All is swell. - } - else { - fail("The job finished with unexpected terminal state " + status + ". " - + "This indicates that there is a bug in the task cancellation."); - } + JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.CANCELED, + flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), + TestingUtils.TESTING_DURATION()); } finally { if (flink != null) { @@ -224,42 +215,6 @@ else if (result instanceof JobNotFound) { } - private JobStatus awaitTermination(ActorGateway jobManager, JobID jobId, FiniteDuration timeout) - throws Exception { - - checkNotNull(jobManager); - checkNotNull(jobId); - checkNotNull(timeout); - - while (true) { - Future ask = jobManager.ask( - new RequestJobStatus(jobId), - timeout); - - Object result = Await.result(ask, timeout); - - if (result instanceof CurrentJobStatus) { - // Success - CurrentJobStatus status = (CurrentJobStatus) result; - - if (!status.jobID().equals(jobId)) { - throw new Exception("JobManager responded for wrong job ID. Request: " - + jobId + ", response: " + status.jobID() + "."); - } - - if (status.status().isTerminalState()) { - return status.status(); - } - } - else if (result instanceof JobNotFound) { - throw new Exception("Cannot find job with ID " + jobId + "."); - } - else { - throw new Exception("Unexpected response to cancel request: " + result); - } - } - } - // --------------------------------------------------------------------------------------------- public static class InfiniteSource extends AbstractInvokable { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 61b1f7aeea793..069b6af1a02ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -18,26 +18,17 @@ package org.apache.flink.runtime.testutils; -import static org.junit.Assert.fail; +import org.apache.flink.runtime.util.FileUtils; -import java.io.BufferedWriter; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.File; -import java.io.FileOutputStream; import java.io.FileWriter; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; +import java.io.InputStream; import java.io.PrintWriter; +import java.io.StringWriter; import java.lang.management.ManagementFactory; import java.lang.management.RuntimeMXBean; - -import org.apache.flink.core.io.IOReadableWritable; -import org.apache.flink.core.memory.InputViewDataInputStreamWrapper; -import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper; +import java.util.UUID; /** * This class contains auxiliary methods for unit tests. @@ -76,6 +67,18 @@ public static String getCurrentClasspath() { return bean.getClassPath(); } + /** + * Create a temporary log4j configuration for the test. + */ + public static File createTemporaryLog4JProperties() throws IOException { + File log4jProps = File.createTempFile(FileUtils.getRandomFilename(""), "-log4j" + + ".properties"); + log4jProps.deleteOnExit(); + CommonTestUtils.printLog4jDebugConfig(log4jProps); + + return log4jProps; + } + /** * Tries to get the java executable command with which the current JVM was started. * Returns null, if the command could not be found. @@ -152,4 +155,50 @@ public static void printLog4jDebugConfig(File file) throws IOException { fw.close(); } } + + public static File createTempDirectory() throws IOException { + File tempDir = new File(System.getProperty("java.io.tmpdir")); + + for (int i = 0; i < 10; i++) { + File dir = new File(tempDir, UUID.randomUUID().toString()); + if (!dir.exists() && dir.mkdirs()) { + return dir; + } + System.err.println("Could not use temporary directory " + dir.getAbsolutePath()); + } + + throw new IOException("Could not create temporary file directory"); + } + + /** + * Utility class to read the output of a process stream and forward it into a StringWriter. + */ + public static class PipeForwarder extends Thread { + + private final StringWriter target; + private final InputStream source; + + public PipeForwarder(InputStream source, StringWriter target) { + super("Pipe Forwarder"); + setDaemon(true); + + this.source = source; + this.target = target; + + start(); + } + + @Override + public void run() { + try { + int next; + while ((next = source.read()) != -1) { + target.write(next); + } + } + catch (IOException e) { + // terminate + } + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java new file mode 100644 index 0000000000000..66e1d9b359f83 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerActorTestUtils.java @@ -0,0 +1,166 @@ +/* + * 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.testutils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.messages.JobManagerMessages.CurrentJobStatus; +import org.apache.flink.runtime.messages.JobManagerMessages.JobStatusResponse; +import org.apache.flink.runtime.testingUtils.TestingJobManager; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.flink.runtime.messages.JobManagerMessages.JobNotFound; +import static org.apache.flink.runtime.messages.JobManagerMessages.RequestJobStatus; +import static org.apache.flink.runtime.messages.JobManagerMessages.getRequestJobStatus; +import static org.apache.flink.runtime.messages.JobManagerMessages.getRequestNumberRegisteredTaskManager; + +/** + * JobManager actor test utilities. + * + *

If you are using a {@link TestingJobManager} most of these are not needed. + */ +public class JobManagerActorTestUtils { + + /** + * Waits for the expected {@link JobStatus}. + * + *

Repeatedly queries the JobManager via {@link RequestJobStatus} messages. + * + * @param jobId Job ID of the job to wait for + * @param expectedJobStatus Expected job status + * @param jobManager Job manager actor to ask + * @param timeout Timeout after which the operation fails + * @throws Exception If the job is not found within the timeout or the job is in another state. + */ + public static void waitForJobStatus( + JobID jobId, + JobStatus expectedJobStatus, + ActorGateway jobManager, + FiniteDuration timeout) throws Exception { + + checkNotNull(jobId, "Job ID"); + checkNotNull(expectedJobStatus, "Expected job status"); + checkNotNull(jobManager, "Job manager"); + checkNotNull(timeout, "Timeout"); + + final Deadline deadline = timeout.fromNow(); + + while (deadline.hasTimeLeft()) { + // Request the job status + JobStatusResponse response = requestJobStatus(jobId, jobManager, deadline.timeLeft()); + + // Found the job + if (response instanceof CurrentJobStatus) { + JobStatus jobStatus = ((CurrentJobStatus) response).status(); + + // OK, that's what we were waiting for + if (jobStatus == expectedJobStatus) { + return; + } + else if (jobStatus.isTerminalState()) { + throw new IllegalStateException("Job is in terminal state " + jobStatus + ", " + + "but was waiting for " + expectedJobStatus + "."); + } + } + // Did not find the job... retry + else if (response instanceof JobNotFound) { + Thread.sleep(Math.min(100, deadline.timeLeft().toMillis())); + } + else { + throw new IllegalStateException("Unexpected response."); + } + } + + throw new IllegalStateException("Job not found within deadline."); + } + + /** + * Request a {@link JobStatusResponse}. + * + * @param jobId Job ID of the job to request the status of + * @param jobManager Job manager actor to ask + * @param timeout Timeout after which the operation fails + * @return The {@link JobStatusResponse} from the job manager + * @throws Exception If there is no answer within the timeout. + */ + public static JobStatusResponse requestJobStatus( + JobID jobId, + ActorGateway jobManager, + FiniteDuration timeout) throws Exception { + + checkNotNull(jobId, "Job ID"); + checkNotNull(jobManager, "Job manager"); + checkNotNull(timeout, "Timeout"); + + // Ask the JobManager + RequestJobStatus request = (RequestJobStatus) getRequestJobStatus(jobId); + Future ask = jobManager.ask(request, timeout); + Object response = Await.result(ask, timeout); + + if (response instanceof JobStatusResponse) { + return (JobStatusResponse) response; + } + + throw new IllegalStateException("Unexpected response."); + } + + /** + * Waits for a minimum number of task managers to connect to the job manager. + * + * @param minimumNumberOfTaskManagers Minimum number of task managers to wait for + * @param jobManager Job manager actor to ask + * @param timeout Timeout after which the operation fails + * @throws Exception If the task managers don't connection with the timeout. + */ + public static void waitForTaskManagers( + int minimumNumberOfTaskManagers, + ActorGateway jobManager, + FiniteDuration timeout) throws Exception { + + checkArgument(minimumNumberOfTaskManagers >= 1); + checkNotNull(jobManager, "Job manager"); + checkNotNull(timeout, "Timeout"); + + final Deadline deadline = timeout.fromNow(); + + while (deadline.hasTimeLeft()) { + Future ask = jobManager.ask(getRequestNumberRegisteredTaskManager(), + deadline.timeLeft()); + + Integer response = (Integer) Await.result(ask, deadline.timeLeft()); + + // All are connected. We are done. + if (response >= minimumNumberOfTaskManagers) { + return; + } + // Waiting for more... retry + else { + Thread.sleep(Math.min(100, deadline.timeLeft().toMillis())); + } + } + + throw new IllegalStateException("Task managers not connected within deadline."); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java new file mode 100644 index 0000000000000..85b768def5e5d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/JobManagerProcess.java @@ -0,0 +1,226 @@ +/* + * 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.testutils; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.jobmanager.JobManager; +import org.apache.flink.runtime.jobmanager.JobManagerMode; +import org.apache.flink.util.NetUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A {@link JobManager} instance running in a separate JVM. + */ +public class JobManagerProcess extends TestJvmProcess { + + private static final Logger LOG = LoggerFactory.getLogger(JobManagerProcess.class); + + /** ID for this JobManager */ + private final int id; + + /** The port the JobManager listens on */ + private final int jobManagerPort; + + /** The configuration for the JobManager */ + private final Configuration config; + + /** Configuration parsed as args for {@link JobManagerProcess.JobManagerProcessEntryPoint} */ + private final String[] jvmArgs; + + private ActorRef jobManagerRef; + + /** + * Creates a {@link JobManager} running in a separate JVM. + * + *

See {@link #JobManagerProcess(int, Configuration, int)} for a more + * detailed + * description. + * + * @param config Configuration for the job manager process + * @throws Exception + */ + public JobManagerProcess(int id, Configuration config) throws Exception { + this(id, config, 0); + } + + /** + * Creates a {@link JobManager} running in a separate JVM. + * + * @param id ID for the JobManager + * @param config Configuration for the job manager process + * @param jobManagerPort Job manager port (if 0, pick any available port) + * @throws Exception + */ + public JobManagerProcess(int id, Configuration config, int jobManagerPort) throws Exception { + checkArgument(id >= 0, "Negative ID"); + this.id = id; + this.config = checkNotNull(config, "Configuration"); + this.jobManagerPort = jobManagerPort <= 0 ? NetUtils.getAvailablePort() : jobManagerPort; + + ArrayList args = new ArrayList<>(); + args.add("--port"); + args.add(String.valueOf(this.jobManagerPort)); + + for (Map.Entry entry : config.toMap().entrySet()) { + args.add("--" + entry.getKey()); + args.add(entry.getValue()); + } + + this.jvmArgs = new String[args.size()]; + args.toArray(jvmArgs); + } + + @Override + public String getName() { + return "JobManager " + id; + } + + @Override + public String[] getJvmArgs() { + return jvmArgs; + } + + @Override + public String getEntryPointClassName() { + return JobManagerProcessEntryPoint.class.getName(); + } + + public int getJobManagerPort() { + return jobManagerPort; + } + + public Configuration getConfig() { + return config; + } + + /** + * Returns the Akka URL of this JobManager. + */ + public String getJobManagerAkkaURL() { + return JobManager.getRemoteJobManagerAkkaURL( + new InetSocketAddress("localhost", jobManagerPort), + Option.empty()); + } + + @Override + public String toString() { + return String.format("JobManagerProcess(id=%d, port=%d)", id, jobManagerPort); + } + + /** + * Waits for the job manager to be reachable. + * + *

Important: Make sure to set the timeout larger than Akka's gating + * time. Otherwise, this will effectively not wait for the JobManager to startup, because the + * time out will fire immediately. + * + * @param actorSystem Actor system to be used to resolve JobManager address. + * @param timeout Timeout (make sure to set larger than Akka's gating time). + */ + public ActorRef getActorRef(ActorSystem actorSystem, FiniteDuration timeout) + throws Exception { + + if (jobManagerRef != null) { + return jobManagerRef; + } + + checkNotNull(actorSystem, "Actor system"); + + // Deadline passes timeout ms + Deadline deadline = timeout.fromNow(); + + while (deadline.hasTimeLeft()) { + try { + // If the Actor is not reachable yet, this throws an Exception. Retry until the + // deadline passes. + this.jobManagerRef = AkkaUtils.getActorRef( + getJobManagerAkkaURL(), + actorSystem, + deadline.timeLeft()); + + return jobManagerRef; + } + catch (Throwable ignored) { + // Retry + Thread.sleep(Math.min(100, deadline.timeLeft().toMillis())); + } + } + + throw new IllegalStateException("JobManager did not start up within " + timeout + "."); + } + + /** + * Entry point for the JobManager process. + */ + public static class JobManagerProcessEntryPoint { + + private static final Logger LOG = LoggerFactory.getLogger(JobManagerProcessEntryPoint.class); + + /** + * Runs the JobManager process in {@link JobManagerMode#CLUSTER} and {@link + * StreamingMode#STREAMING} (can handle both batch and streaming jobs). + * + *

Required argument: port. Start the process with + * --port PORT. + * + *

Other arguments are parsed to a {@link Configuration} and passed to the + * JobManager, for instance: --recovery.mode ZOOKEEPER --ha.zookeeper.quorum + * "xyz:123:456". + */ + public static void main(String[] args) { + try { + ParameterTool params = ParameterTool.fromArgs(args); + final int port = Integer.valueOf(params.getRequired("port")); + LOG.info("Running on port {}.", port); + + Configuration config = params.getConfiguration(); + LOG.info("Configuration: {}.", config); + + // Run the JobManager + JobManager.runJobManager(config, JobManagerMode.CLUSTER, StreamingMode.STREAMING, + "localhost", port); + + // Run forever. Forever, ever? Forever, ever! + new CountDownLatch(1).await(); + } + catch (Throwable t) { + LOG.error("Failed to start JobManager process", t); + System.exit(1); + } + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java new file mode 100644 index 0000000000000..f683c558d0248 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TaskManagerProcess.java @@ -0,0 +1,133 @@ +/* + * 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.testutils; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Map; +import java.util.concurrent.CountDownLatch; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * A {@link TaskManager} instance running in a separate JVM. + */ +public class TaskManagerProcess extends TestJvmProcess { + + /** ID for this TaskManager */ + private final int id; + + /** The configuration for the TaskManager */ + private final Configuration config; + + /** Configuration parsed as args for {@link TaskManagerProcess.TaskManagerProcessEntryPoint} */ + private final String[] jvmArgs; + + public TaskManagerProcess(int id, Configuration config) throws Exception { + checkArgument(id >= 0, "Negative ID"); + this.id = id; + this.config = checkNotNull(config, "Configuration"); + + ArrayList args = new ArrayList<>(); + + for (Map.Entry entry : config.toMap().entrySet()) { + args.add("--" + entry.getKey()); + args.add(entry.getValue()); + } + + this.jvmArgs = new String[args.size()]; + args.toArray(jvmArgs); + } + + @Override + public String getName() { + return "TaskManager " + id; + } + + @Override + public String[] getJvmArgs() { + return jvmArgs; + } + + @Override + public String getEntryPointClassName() { + return TaskManagerProcessEntryPoint.class.getName(); + } + + public int getId() { + return id; + } + + @Override + public String toString() { + return String.format("TaskManagerProcess(id=%d)", id); + } + + /** + * Entry point for the TaskManager process. + */ + public static class TaskManagerProcessEntryPoint { + + private static final Logger LOG = LoggerFactory.getLogger(TaskManagerProcessEntryPoint.class); + + /** + * Runs the JobManager process in {@link StreamingMode#STREAMING} (can handle both batch + * and streaming jobs). + * + *

All arguments are parsed to a {@link Configuration} and passed to the Taskmanager, + * for instance: --recovery.mode ZOOKEEPER --ha.zookeeper.quorum "xyz:123:456". + */ + public static void main(String[] args) throws Exception { + try { + Configuration config = ParameterTool.fromArgs(args).getConfiguration(); + + if (!config.containsKey(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY)) { + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4); + } + + if (!config.containsKey(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY)) { + config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100); + } + + + LOG.info("Configuration: {}.", config); + + // Run the TaskManager + TaskManager.selectNetworkInterfaceAndRunTaskManager( + config, StreamingMode.STREAMING, TaskManager.class); + + // Run forever + new CountDownLatch(1).await(); + } + catch (Throwable t) { + LOG.error("Failed to start TaskManager process", t); + System.exit(1); + } + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java new file mode 100644 index 0000000000000..0920b5cb6f869 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestJvmProcess.java @@ -0,0 +1,267 @@ +/* + * 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.testutils; + +import org.apache.commons.lang3.ArrayUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.StringWriter; +import java.util.Arrays; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.flink.runtime.testutils.CommonTestUtils.createTemporaryLog4JProperties; +import static org.apache.flink.runtime.testutils.CommonTestUtils.getCurrentClasspath; +import static org.apache.flink.runtime.testutils.CommonTestUtils.getJavaCommandPath; +import static org.junit.Assert.fail; + +/** + * A {@link Process} running a separate JVM. + */ +public abstract class TestJvmProcess { + + private static final Logger LOG = LoggerFactory.getLogger(TestJvmProcess.class); + + /** Lock to guard {@link #createAndStart()} and {@link #destroy()} calls. */ + private final Object createDestroyLock = new Object(); + + /** The java command path */ + private final String javaCommandPath; + + /** The log4j configuration path. */ + private final String log4jConfigFilePath; + + /** Shutdown hook for resource cleanup */ + private final Thread shutdownHook; + + /** JVM process memory (set for both '-Xms' and '-Xmx'). */ + private int jvmMemoryInMb = 80; + + /** The JVM process */ + private Process process; + + /** Writer for the process output */ + private volatile StringWriter processOutput; + + public TestJvmProcess() throws Exception { + this(getJavaCommandPath(), createTemporaryLog4JProperties().getPath()); + } + + public TestJvmProcess(String javaCommandPath, String log4jConfigFilePath) { + this.javaCommandPath = checkNotNull(javaCommandPath, "Java command path"); + this.log4jConfigFilePath = checkNotNull(log4jConfigFilePath, "log4j config file path"); + + this.shutdownHook = new Thread(new Runnable() { + @Override + public void run() { + try { + destroy(); + } + catch (Throwable t) { + LOG.error("Error during process cleanup shutdown hook.", t); + } + } + }); + } + + /** + * Returns the name of the process. + */ + public abstract String getName(); + + /** + * Returns the arguments to the JVM. + * + *

These can be parsed by the main method of the entry point class. + */ + public abstract String[] getJvmArgs(); + + /** + * Returns the name of the class to run. + * + *

Arguments to the main method can be specified via {@link #getJvmArgs()}. + */ + public abstract String getEntryPointClassName(); + + // --------------------------------------------------------------------------------------------- + + /** + * Sets the memory for the process (-Xms and -Xmx flags) (>= 80). + * + * @param jvmMemoryInMb Amount of memory in Megabytes for the JVM (>= 80). + */ + public void setJVMMemory(int jvmMemoryInMb) { + checkArgument(jvmMemoryInMb >= 80, "JobManager JVM Requires at least 80 MBs of memory."); + this.jvmMemoryInMb = jvmMemoryInMb; + } + + /** + * Creates and starts the {@link Process}. + * + * Important: Don't forget to call {@link #destroy()} to prevent + * resource leaks. The created process will be child process and is not guaranteed to + * terminate when the parent process terminates. + */ + public void createAndStart() throws IOException { + String[] cmd = new String[] { + javaCommandPath, + "-Dlog.level=DEBUG", + "-Dlog4j.configuration=file:" + log4jConfigFilePath, + "-Xms" + jvmMemoryInMb + "m", + "-Xmx" + jvmMemoryInMb + "m", + "-classpath", getCurrentClasspath(), + getEntryPointClassName() }; + + String[] jvmArgs = getJvmArgs(); + + if (jvmArgs != null && jvmArgs.length > 0) { + cmd = ArrayUtils.addAll(cmd, jvmArgs); + } + + synchronized (createDestroyLock) { + if (process == null) { + LOG.debug("Running command '{}'.", Arrays.toString(cmd)); + this.process = new ProcessBuilder(cmd).start(); + + // Forward output + this.processOutput = new StringWriter(); + new CommonTestUtils.PipeForwarder(process.getErrorStream(), processOutput); + + try { + // Add JVM shutdown hook to call shutdown of service + Runtime.getRuntime().addShutdownHook(shutdownHook); + } + catch (IllegalStateException ignored) { + // JVM is already shutting down. No need to do this. + } + catch (Throwable t) { + LOG.error("Cannot register process cleanup shutdown hook.", t); + } + } + else { + throw new IllegalStateException("Already running."); + } + } + } + + public void printProcessLog() { + if (processOutput == null) { + throw new IllegalStateException("Not started"); + } + + System.out.println("-----------------------------------------"); + System.out.println(" BEGIN SPAWNED PROCESS LOG FOR " + getName()); + System.out.println("-----------------------------------------"); + + String out = processOutput.toString(); + if (out == null || out.length() == 0) { + System.out.println("(EMPTY)"); + } + else { + System.out.println(out); + } + + System.out.println("-----------------------------------------"); + System.out.println(" END SPAWNED PROCESS LOG " + getName()); + System.out.println("-----------------------------------------"); + } + + public void destroy() { + synchronized (createDestroyLock) { + if (process != null) { + LOG.debug("Destroying " + getName() + " process."); + + try { + process.destroy(); + } + catch (Throwable t) { + LOG.error("Error while trying to destroy process.", t); + } + finally { + process = null; + + if (shutdownHook != null && shutdownHook != Thread.currentThread()) { + try { + Runtime.getRuntime().removeShutdownHook(shutdownHook); + } + catch (IllegalStateException ignored) { + // JVM is in shutdown already, we can safely ignore this. + } + catch (Throwable t) { + LOG.warn("Exception while unregistering prcess cleanup shutdown hook."); + } + } + } + } + } + } + + // --------------------------------------------------------------------------------------------- + // File based synchronization utilities + // --------------------------------------------------------------------------------------------- + + public static void touchFile(File file) throws IOException { + if (!file.exists()) { + new FileOutputStream(file).close(); + } + if (!file.setLastModified(System.currentTimeMillis())) { + throw new IOException("Could not touch the file."); + } + } + + public static void waitForMarkerFiles(File basedir, String prefix, int num, long timeout) { + long now = System.currentTimeMillis(); + final long deadline = now + timeout; + + + while (now < deadline) { + boolean allFound = true; + + for (int i = 0; i < num; i++) { + File nextToCheck = new File(basedir, prefix + i); + if (!nextToCheck.exists()) { + allFound = false; + break; + } + } + + if (allFound) { + return; + } + else { + // not all found, wait for a bit + try { + Thread.sleep(10); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + + now = System.currentTimeMillis(); + } + } + + fail("The tasks were not started within time (" + timeout + "msecs)"); + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java new file mode 100644 index 0000000000000..d2e5b6ad4d17e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java @@ -0,0 +1,94 @@ +/* + * 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.testutils; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.RecoveryMode; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * ZooKeeper test utilities. + */ +public class ZooKeeperTestUtils { + + /** + * Creates a configuration to operate in {@link RecoveryMode#ZOOKEEPER}. + * + * @param zooKeeperQuorum ZooKeeper quorum to connect to + * @param fsStateHandlePath Base path for file system state backend (for checkpoints and + * recovery) + * @return A new configuration to operate in {@link RecoveryMode#ZOOKEEPER}. + */ + public static Configuration createZooKeeperRecoveryModeConfig( + String zooKeeperQuorum, String fsStateHandlePath) { + + return setZooKeeperRecoveryMode(new Configuration(), zooKeeperQuorum, fsStateHandlePath); + } + + /** + * Sets all necessary configuration keys to operate in {@link RecoveryMode#ZOOKEEPER}. + * + * @param config Configuration to use + * @param zooKeeperQuorum ZooKeeper quorum to connect to + * @param fsStateHandlePath Base path for file system state backend (for checkpoints and + * recovery) + * @return The modified configuration to operate in {@link RecoveryMode#ZOOKEEPER}. + */ + public static Configuration setZooKeeperRecoveryMode( + Configuration config, + String zooKeeperQuorum, + String fsStateHandlePath) { + + checkNotNull(config, "Configuration"); + checkNotNull(zooKeeperQuorum, "ZooKeeper quorum"); + checkNotNull(fsStateHandlePath, "File state handle backend path"); + + // Web frontend, you have been dismissed. Sorry. + config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1); + + // ZooKeeper recovery mode + config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER"); + config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zooKeeperQuorum); + + int connTimeout = 5000; + if (System.getenv().get("CI") != null) { + // The regular timeout is to aggressive for Travis and connections are often lost. + connTimeout = 20000; + } + + config.setInteger(ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout); + config.setInteger(ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT, connTimeout); + + // File system state backend + config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM"); + config.setString(ConfigConstants.STATE_BACKEND_FS_DIR, fsStateHandlePath + "/checkpoints"); + config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, fsStateHandlePath + "/recovery"); + + // Akka failure detection and execution retries + config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms"); + config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s"); + config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9); + config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "10 s"); + + return config; + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java new file mode 100644 index 0000000000000..f0130ec228e6a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreITCase.java @@ -0,0 +1,591 @@ +/* + * 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.zookeeper; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.api.BackgroundCallback; +import org.apache.curator.framework.api.CuratorEvent; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.state.StateHandle; +import org.apache.flink.runtime.state.StateHandleProvider; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.TestLogger; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.data.Stat; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for basic {@link ZooKeeperStateHandleStore} behaviour. + * + *

Tests include: + *

    + *
  • Expected usage of operations
  • + *
  • Correct ordering of ZooKeeper and state handle operations
  • + *
+ */ +public class ZooKeeperStateHandleStoreITCase extends TestLogger { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + } + + /** + * Tests add operation with default {@link CreateMode}. + */ + @Test + public void testAdd() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testAdd"; + final Long state = 1239712317L; + + // Test + store.add(pathInZooKeeper, state); + + // Verify + // State handle created + assertEquals(1, stateHandleProvider.getStateHandles().size()); + assertEquals(state, stateHandleProvider.getStateHandles().get(0).getState(null)); + + // Path created and is persistent + Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper); + assertNotNull(stat); + assertEquals(0, stat.getEphemeralOwner()); + + // Data is equal + @SuppressWarnings("unchecked") + Long actual = ((StateHandle) InstantiationUtil.deserializeObject( + ZooKeeper.getClient().getData().forPath(pathInZooKeeper), + ClassLoader.getSystemClassLoader())).getState(null); + + assertEquals(state, actual); + } + + /** + * Tests that {@link CreateMode} is respected. + */ + @Test + public void testAddWithCreateMode() throws Exception { + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + Long state = 3457347234L; + + CreateMode[] modes = CreateMode.values(); + for (int i = 0; i < modes.length; i++) { + CreateMode mode = modes[i]; + state += i; + + String pathInZooKeeper = "/testAddWithCreateMode" + mode.name(); + + // Test + store.add(pathInZooKeeper, state, mode); + + if (mode.isSequential()) { + // Figure out the sequential ID + List paths = ZooKeeper.getClient().getChildren().forPath("/"); + for (String p : paths) { + if (p.startsWith("testAddWithCreateMode" + mode.name())) { + pathInZooKeeper = "/" + p; + break; + } + } + } + + // Verify + // State handle created + assertEquals(i + 1, stateHandleProvider.getStateHandles().size()); + assertEquals(state, stateHandleProvider.getStateHandles().get(i).getState(null)); + + // Path created + Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper); + + assertNotNull(stat); + + // Is ephemeral or persistent + if (mode.isEphemeral()) { + assertTrue(stat.getEphemeralOwner() != 0); + } + else { + assertEquals(0, stat.getEphemeralOwner()); + } + + // Data is equal + @SuppressWarnings("unchecked") + Long actual = ((StateHandle) InstantiationUtil.deserializeObject( + ZooKeeper.getClient().getData().forPath(pathInZooKeeper), + ClassLoader.getSystemClassLoader())).getState(null); + + assertEquals(state, actual); + } + } + + /** + * Tests that an existing path throws an Exception. + */ + @Test(expected = Exception.class) + public void testAddAlreadyExistingPath() throws Exception { + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + ZooKeeper.getClient().create().forPath("/testAddAlreadyExistingPath"); + + store.add("/testAddAlreadyExistingPath", 1L); + } + + /** + * Tests that the created state handle is discarded if ZooKeeper create fails. + */ + @Test + public void testAddDiscardStateHandleAfterFailure() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + CuratorFramework client = spy(ZooKeeper.getClient()); + when(client.create()).thenThrow(new RuntimeException("Expected test Exception.")); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + client, stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testAddDiscardStateHandleAfterFailure"; + final Long state = 81282227L; + + try { + // Test + store.add(pathInZooKeeper, state); + fail("Did not throw expected exception"); + } + catch (Exception ignored) { + } + + // Verify + // State handle created and discarded + assertEquals(1, stateHandleProvider.getStateHandles().size()); + assertEquals(state, stateHandleProvider.getStateHandles().get(0).getState(null)); + assertEquals(1, stateHandleProvider.getStateHandles().get(0).getNumberOfDiscardCalls()); + } + + /** + * Tests that a state handle is replaced. + */ + @Test + public void testReplace() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testReplace"; + final Long initialState = 30968470898L; + final Long replaceState = 88383776661L; + + // Test + store.add(pathInZooKeeper, initialState); + store.replace(pathInZooKeeper, 0, replaceState); + + // Verify + // State handles created + assertEquals(2, stateHandleProvider.getStateHandles().size()); + assertEquals(initialState, stateHandleProvider.getStateHandles().get(0).getState(null)); + assertEquals(replaceState, stateHandleProvider.getStateHandles().get(1).getState(null)); + + // Path created and is persistent + Stat stat = ZooKeeper.getClient().checkExists().forPath(pathInZooKeeper); + assertNotNull(stat); + assertEquals(0, stat.getEphemeralOwner()); + + // Data is equal + @SuppressWarnings("unchecked") + Long actual = ((StateHandle) InstantiationUtil.deserializeObject( + ZooKeeper.getClient().getData().forPath(pathInZooKeeper), + ClassLoader.getSystemClassLoader())).getState(null); + + assertEquals(replaceState, actual); + } + + /** + * Tests that a non existing path throws an Exception. + */ + @Test(expected = Exception.class) + public void testReplaceNonExistingPath() throws Exception { + StateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + store.replace("/testReplaceNonExistingPath", 0, 1L); + } + + /** + * Tests that the replace state handle is discarded if ZooKeeper setData fails. + */ + @Test + public void testReplaceDiscardStateHandleAfterFailure() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + CuratorFramework client = spy(ZooKeeper.getClient()); + when(client.setData()).thenThrow(new RuntimeException("Expected test Exception.")); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + client, stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testReplaceDiscardStateHandleAfterFailure"; + final Long initialState = 30968470898L; + final Long replaceState = 88383776661L; + + // Test + store.add(pathInZooKeeper, initialState); + + try { + store.replace(pathInZooKeeper, 0, replaceState); + fail("Did not throw expected exception"); + } + catch (Exception ignored) { + } + + // Verify + // State handle created and discarded + assertEquals(2, stateHandleProvider.getStateHandles().size()); + assertEquals(initialState, stateHandleProvider.getStateHandles().get(0).getState(null)); + assertEquals(replaceState, stateHandleProvider.getStateHandles().get(1).getState(null)); + assertEquals(1, stateHandleProvider.getStateHandles().get(1).getNumberOfDiscardCalls()); + + // Initial value + @SuppressWarnings("unchecked") + Long actual = ((StateHandle) InstantiationUtil.deserializeObject( + ZooKeeper.getClient().getData().forPath(pathInZooKeeper), + ClassLoader.getSystemClassLoader())).getState(null); + + assertEquals(initialState, actual); + } + + /** + * Tests get operation. + */ + @Test + public void testGetAndExists() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testGetAndExists"; + final Long state = 311222268470898L; + + // Test + assertEquals(-1, store.exists(pathInZooKeeper)); + + store.add(pathInZooKeeper, state); + StateHandle actual = store.get(pathInZooKeeper); + + // Verify + assertEquals(state, actual.getState(null)); + assertTrue(store.exists(pathInZooKeeper) >= 0); + } + + /** + * Tests that a non existing path throws an Exception. + */ + @Test(expected = Exception.class) + public void testGetNonExistingPath() throws Exception { + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + store.get("/testGetNonExistingPath"); + } + + /** + * Tests that all added state is returned. + */ + @Test + public void testGetAll() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testGetAll"; + + final Set expected = new HashSet<>(); + expected.add(311222268470898L); + expected.add(132812888L); + expected.add(27255442L); + expected.add(11122233124L); + + // Test + for (long val : expected) { + store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL); + } + + for (Tuple2, String> val : store.getAll()) { + assertTrue(expected.remove(val.f0.getState(null))); + } + assertEquals(0, expected.size()); + } + + /** + * Tests that the state is returned sorted. + */ + @Test + public void testGetAllSortedByName() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testGetAllSortedByName"; + + final Long[] expected = new Long[] { + 311222268470898L, 132812888L, 27255442L, 11122233124L }; + + // Test + for (long val : expected) { + store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL); + } + + List, String>> actual = store.getAllSortedByName(); + assertEquals(expected.length, actual.size()); + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], actual.get(i).f0.getState(null)); + } + } + + /** + * Tests that state handles are correctly removed. + */ + @Test + public void testRemove() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testRemove"; + final Long state = 27255442L; + + store.add(pathInZooKeeper, state); + + // Test + store.remove(pathInZooKeeper); + + // Verify discarded + assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size()); + } + + /** + * Tests that state handles are correctly removed with a callback. + */ + @Test + public void testRemoveWithCallback() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testRemoveWithCallback"; + final Long state = 27255442L; + + store.add(pathInZooKeeper, state); + + final CountDownLatch sync = new CountDownLatch(1); + BackgroundCallback callback = mock(BackgroundCallback.class); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + sync.countDown(); + return null; + } + }).when(callback).processResult(eq(ZooKeeper.getClient()), any(CuratorEvent.class)); + + // Test + store.remove(pathInZooKeeper, callback); + + // Verify discarded and callback called + assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size()); + + sync.await(); + + verify(callback, times(1)) + .processResult(eq(ZooKeeper.getClient()), any(CuratorEvent.class)); + } + + /** + * Tests that state handles are correctly discarded. + */ + @Test + public void testRemoveAndDiscardState() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testDiscard"; + final Long state = 27255442L; + + store.add(pathInZooKeeper, state); + + // Test + store.removeAndDiscardState(pathInZooKeeper); + + // Verify discarded + assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size()); + } + + /** Tests that all state handles are correctly discarded. */ + @Test + public void testRemoveAndDiscardAllState() throws Exception { + // Setup + LongStateHandleProvider stateHandleProvider = new LongStateHandleProvider(); + + ZooKeeperStateHandleStore store = new ZooKeeperStateHandleStore<>( + ZooKeeper.getClient(), stateHandleProvider); + + // Config + final String pathInZooKeeper = "/testDiscardAll"; + + final Set expected = new HashSet<>(); + expected.add(311222268470898L); + expected.add(132812888L); + expected.add(27255442L); + expected.add(11122233124L); + + // Test + for (long val : expected) { + store.add(pathInZooKeeper, val, CreateMode.PERSISTENT_SEQUENTIAL); + } + + store.removeAndDiscardAllState(); + + // Verify all discarded + assertEquals(0, ZooKeeper.getClient().getChildren().forPath("/").size()); + } + + // --------------------------------------------------------------------------------------------- + // Simple test helpers + // --------------------------------------------------------------------------------------------- + + private static class LongStateHandleProvider implements StateHandleProvider { + + private static final long serialVersionUID = 4572084854499402276L; + + private final List stateHandles = new ArrayList<>(); + + @Override + public StateHandle createStateHandle(Long state) { + LongStateHandle stateHandle = new LongStateHandle(state); + stateHandles.add(stateHandle); + + return stateHandle; + } + + public List getStateHandles() { + return stateHandles; + } + } + + private static class LongStateHandle implements StateHandle { + + private static final long serialVersionUID = -3555329254423838912L; + + private final Long state; + + private int numberOfDiscardCalls; + + public LongStateHandle(Long state) { + this.state = state; + } + + @Override + public Long getState(ClassLoader ignored) throws Exception { + return state; + } + + @Override + public void discardState() throws Exception { + numberOfDiscardCalls++; + } + + public int getNumberOfDiscardCalls() { + return numberOfDiscardCalls; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java new file mode 100644 index 0000000000000..7ae89d16f66b0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java @@ -0,0 +1,133 @@ +/* + * 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.zookeeper; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.test.TestingCluster; +import org.apache.curator.test.TestingServer; +import org.apache.curator.utils.ZKPaths; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +/** + * Simple ZooKeeper and CuratorFramework setup for tests. + */ +public class ZooKeeperTestEnvironment { + + private final TestingServer zooKeeperServer; + + private final TestingCluster zooKeeperCluster; + + private final CuratorFramework client; + + /** + * Starts a ZooKeeper cluster with the number of quorum peers and a client. + * + * @param numberOfZooKeeperQuorumPeers Starts a {@link TestingServer}, if 1. + * Starts a {@link TestingCluster}, if =>1. + */ + public ZooKeeperTestEnvironment(int numberOfZooKeeperQuorumPeers) { + if (numberOfZooKeeperQuorumPeers <= 0) { + throw new IllegalArgumentException("Number of peers needs to be >= 1."); + } + + final Configuration conf = new Configuration(); + + try { + if (numberOfZooKeeperQuorumPeers == 1) { + zooKeeperServer = new TestingServer(true); + zooKeeperCluster = null; + + conf.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, + zooKeeperServer.getConnectString()); + } + else { + zooKeeperServer = null; + zooKeeperCluster = new TestingCluster(numberOfZooKeeperQuorumPeers); + + zooKeeperCluster.start(); + + conf.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, + zooKeeperCluster.getConnectString()); + } + + client = ZooKeeperUtils.startCuratorFramework(conf); + + client.newNamespaceAwareEnsurePath("/") + .ensure(client.getZookeeperClient()); + } + catch (Exception e) { + throw new RuntimeException("Error setting up ZooKeeperTestEnvironment", e); + } + } + + /** + * Shutdown the client and ZooKeeper server/cluster. + */ + public void shutdown() throws Exception { + if (client != null) { + client.close(); + } + + if (zooKeeperServer != null) { + zooKeeperServer.close(); + } + + if (zooKeeperCluster != null) { + zooKeeperCluster.close(); + } + } + + public String getConnectString() { + if (zooKeeperServer != null) { + return zooKeeperServer.getConnectString(); + } + else { + return zooKeeperCluster.getConnectString(); + } + } + + /** + * Returns a client for the started ZooKeeper server/cluster. + */ + public CuratorFramework getClient() { + return client; + } + + /** + * Creates a new client for the started ZooKeeper server/cluster. + */ + public CuratorFramework createClient() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, getConnectString()); + return ZooKeeperUtils.startCuratorFramework(config); + } + + /** + * Deletes all ZNodes under the root node. + * + * @throws Exception If the ZooKeeper operation fails + */ + public void deleteAll() throws Exception { + final String path = "/" + client.getNamespace(); + ZKPaths.deleteChildren(client.getZookeeperClient().getZooKeeper(), path, false); + } + +} diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala index d1b8facacb513..9a1cde0234779 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.scala @@ -18,9 +18,12 @@ package org.apache.flink.runtime.executiongraph +import java.util.concurrent.TimeUnit + import org.apache.flink.api.common.JobID import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.execution.ExecutionState import org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.SimpleActorGateway import org.apache.flink.runtime.jobgraph.{JobStatus, JobGraph, JobVertex} import org.apache.flink.runtime.jobmanager.Tasks @@ -32,6 +35,7 @@ import org.scalatest.junit.JUnitRunner import org.scalatest.{Matchers, WordSpecLike} import scala.collection.JavaConverters._ +import scala.concurrent.duration.FiniteDuration @RunWith(classOf[JUnitRunner]) class ExecutionGraphRestartTest extends WordSpecLike with Matchers { @@ -126,8 +130,23 @@ class ExecutionGraphRestartTest extends WordSpecLike with Matchers { for (vertex <- eg.getAllExecutionVertices.asScala) { vertex.getCurrentExecutionAttempt().cancelingComplete() } - + + val timeout = new FiniteDuration(2, TimeUnit.MINUTES) + + // Wait for async restart + var deadline = timeout.fromNow + while (deadline.hasTimeLeft() && eg.getState != JobStatus.RUNNING) { + Thread.sleep(100) + } + eg.getState should equal(JobStatus.RUNNING) + + // Wait for deploying after async restart + deadline = timeout.fromNow + while (deadline.hasTimeLeft() && eg.getAllExecutionVertices.asScala.exists( + _.getCurrentExecutionAttempt.getState != ExecutionState.DEPLOYING)) { + Thread.sleep(100) + } for (vertex <- eg.getAllExecutionVertices.asScala) { vertex.getCurrentExecutionAttempt().markFinished() diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index c9ae1e4f82ed8..703d7bfe575ae 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -43,28 +43,28 @@ import scala.concurrent.{Await, Future} * otherwise false */ class TestingCluster( - userConfiguration: Configuration, - singleActorSystem: Boolean, - synchronousDispatcher: Boolean, - streamingMode: StreamingMode) + userConfiguration: Configuration, + singleActorSystem: Boolean, + synchronousDispatcher: Boolean, + streamingMode: StreamingMode) extends FlinkMiniCluster( userConfiguration, singleActorSystem, streamingMode) { - + def this(userConfiguration: Configuration, singleActorSystem: Boolean, synchronousDispatcher: Boolean) - = this(userConfiguration, singleActorSystem, synchronousDispatcher, StreamingMode.BATCH_ONLY) + = this(userConfiguration, singleActorSystem, synchronousDispatcher, StreamingMode.BATCH_ONLY) def this(userConfiguration: Configuration, singleActorSystem: Boolean) - = this(userConfiguration, singleActorSystem, false) + = this(userConfiguration, singleActorSystem, false) def this(userConfiguration: Configuration) = this(userConfiguration, true, false) - + // -------------------------------------------------------------------------- - + override def generateConfiguration(userConfig: Configuration): Configuration = { val cfg = new Configuration() cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost") @@ -100,16 +100,18 @@ class TestingCluster( } val (executionContext, - instanceManager, - scheduler, - libraryCacheManager, - executionRetries, - delayBetweenRetries, - timeout, - archiveCount, - leaderElectionService) = JobManager.createJobManagerComponents( - config, - createLeaderElectionService()) + instanceManager, + scheduler, + libraryCacheManager, + executionRetries, + delayBetweenRetries, + timeout, + archiveCount, + leaderElectionService, + submittedJobsGraphs, + checkpointRecoveryFactory) = JobManager.createJobManagerComponents( + config, + createLeaderElectionService()) val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount)) val archive = actorSystem.actorOf(testArchiveProps, archiveName) @@ -126,7 +128,9 @@ class TestingCluster( delayBetweenRetries, timeout, streamingMode, - leaderElectionService)) + leaderElectionService, + submittedJobsGraphs, + checkpointRecoveryFactory)) val dispatcherJobManagerProps = if (synchronousDispatcher) { // disable asynchronous futures (e.g. accumulator update in Heartbeat) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index 312a1e5e4d2e2..be720038dcad3 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -18,32 +18,18 @@ package org.apache.flink.runtime.testingUtils -import akka.actor.{Cancellable, Terminated, ActorRef} -import akka.pattern.pipe -import akka.pattern.ask -import org.apache.flink.api.common.JobID +import akka.actor.ActorRef import org.apache.flink.configuration.Configuration -import org.apache.flink.runtime.leaderelection.LeaderElectionService -import org.apache.flink.runtime.{StreamingMode, FlinkActor} import org.apache.flink.runtime.StreamingMode -import org.apache.flink.runtime.execution.ExecutionState +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.instance.InstanceManager -import org.apache.flink.runtime.jobgraph.JobStatus -import org.apache.flink.runtime.jobmanager.JobManager import org.apache.flink.runtime.jobmanager.scheduler.Scheduler -import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged -import org.apache.flink.runtime.messages.JobManagerMessages.GrantLeadership -import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect} -import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._ -import org.apache.flink.runtime.testingUtils.TestingMessages.{CheckIfJobRemoved, Alive, -DisableDisconnect} -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged +import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore} +import org.apache.flink.runtime.leaderelection.LeaderElectionService -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.ExecutionContext import scala.concurrent.duration._ - import scala.language.postfixOps /** JobManager implementation extended by testing messages @@ -70,7 +56,9 @@ class TestingJobManager( delayBetweenRetries: Long, timeout: FiniteDuration, mode: StreamingMode, - leaderElectionService: LeaderElectionService) + leaderElectionService: LeaderElectionService, + submittedJobGraphs : SubmittedJobGraphStore, + checkpointRecoveryFactory : CheckpointRecoveryFactory) extends JobManager( flinkConfiguration, executionContext, @@ -82,5 +70,7 @@ class TestingJobManager( delayBetweenRetries, timeout, mode, - leaderElectionService) + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) with TestingJobManagerLike {} diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 16b8f55f0d9b7..c63e2e6e68655 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.state.FileStateHandle; import org.apache.flink.runtime.state.LocalStateHandle; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateHandle; import org.apache.flink.runtime.state.StateHandleProvider; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; @@ -458,10 +459,6 @@ private StateHandleProvider getStateHandleProvider() { } } - private enum StateBackend { - JOBMANAGER, FILESYSTEM - } - /** * Registers a timer. */ diff --git a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala index c7f76980bc191..11eb174a0b073 100644 --- a/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala +++ b/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala @@ -20,22 +20,20 @@ package org.apache.flink.test.util import java.util.concurrent.TimeoutException -import akka.pattern.ask -import akka.actor.{Props, ActorRef, ActorSystem} +import akka.actor.{ActorRef, ActorSystem} import akka.pattern.Patterns._ +import akka.pattern.ask import org.apache.curator.test.TestingCluster import org.apache.flink.configuration.{ConfigConstants, Configuration} import org.apache.flink.runtime.StreamingMode import org.apache.flink.runtime.akka.AkkaUtils -import org.apache.flink.runtime.jobmanager.{RecoveryMode, JobManager} +import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode} import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster import org.apache.flink.runtime.taskmanager.TaskManager -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages -.NotifyWhenRegisteredAtJobManager -import org.apache.flink.runtime.testingUtils.{TestingUtils, TestingTaskManager, -TestingJobManager, TestingMemoryArchivist} +import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager +import org.apache.flink.runtime.testingUtils.{TestingJobManager, TestingMemoryArchivist, TestingTaskManager, TestingUtils} -import scala.concurrent.{Future, Await} +import scala.concurrent.{Await, Future} /** * A forkable mini cluster is a special case of the mini cluster, used for parallel test execution @@ -47,20 +45,20 @@ import scala.concurrent.{Future, Await} * same [[ActorSystem]], otherwise false. */ class ForkableFlinkMiniCluster( - userConfiguration: Configuration, - singleActorSystem: Boolean, - streamingMode: StreamingMode) + userConfiguration: Configuration, + singleActorSystem: Boolean, + streamingMode: StreamingMode) extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem, streamingMode) { - def this(userConfiguration: Configuration, singleActorSystem: Boolean) - = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY) + def this(userConfiguration: Configuration, singleActorSystem: Boolean) + = this(userConfiguration, singleActorSystem, StreamingMode.BATCH_ONLY) def this(userConfiguration: Configuration) = this(userConfiguration, true) - + // -------------------------------------------------------------------------- var zookeeperCluster: Option[TestingCluster] = None - + override def generateConfiguration(userConfiguration: Configuration): Configuration = { val forNumberString = System.getProperty("forkNumber") @@ -264,10 +262,10 @@ object ForkableFlinkMiniCluster { import org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT def startCluster( - numSlots: Int, - numTaskManagers: Int, - timeout: String = DEFAULT_AKKA_ASK_TIMEOUT) - : ForkableFlinkMiniCluster = { + numSlots: Int, + numTaskManagers: Int, + timeout: String = DEFAULT_AKKA_ASK_TIMEOUT) + : ForkableFlinkMiniCluster = { val config = new Configuration() config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java similarity index 99% rename from flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java index 8a75de54d6763..29e31fc28a710 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java @@ -55,9 +55,9 @@ * results without testing the checkpointing). */ @SuppressWarnings("serial") -public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase { +public class StateCheckpointedITCase extends StreamFaultToleranceTestBase { - private static final Logger LOG = LoggerFactory.getLogger(StateCheckpoinedITCase.class); + private static final Logger LOG = LoggerFactory.getLogger(StateCheckpointedITCase.class); final long NUM_STRINGS = 10_000_000L; diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java new file mode 100644 index 0000000000000..ba5ff1c11a185 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractJobManagerProcessFailureRecoveryITCase.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.recovery; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.commons.io.FileUtils; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.leaderelection.TestingListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; +import org.apache.flink.runtime.testutils.JobManagerProcess; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import scala.Option; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.runtime.testutils.CommonTestUtils.createTempDirectory; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; + +/** + * Verify behaviour in case of JobManager process failure during job execution. + * + *

The test works with multiple job managers processes by spawning JVMs. + * + *

Initially, it starts two TaskManager (2 slots each) and two JobManager JVMs. + * + *

It submits a program with parallelism 4 and waits until all tasks are brought up. + * Coordination between the test and the tasks happens via checking for the existence of + * temporary files. It then kills the leading JobManager process. The recovery should restart the + * tasks on the new JobManager. + * + *

This follows the same structure as {@link AbstractTaskManagerProcessFailureRecoveryTest}. + */ +public abstract class AbstractJobManagerProcessFailureRecoveryITCase extends TestLogger { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES); + + private static final File FileStateBackendBasePath; + + static { + try { + FileStateBackendBasePath = CommonTestUtils.createTempDirectory(); + } + catch (IOException e) { + throw new RuntimeException("Error in test setup. Could not create directory.", e); + } + } + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + + if (FileStateBackendBasePath != null) { + FileUtils.deleteDirectory(FileStateBackendBasePath); + } + } + + @Before + public void cleanUp() throws Exception { + ZooKeeper.deleteAll(); + + FileUtils.cleanDirectory(FileStateBackendBasePath); + } + + protected static final String READY_MARKER_FILE_PREFIX = "ready_"; + protected static final String FINISH_MARKER_FILE_PREFIX = "finish_"; + protected static final String PROCEED_MARKER_FILE = "proceed"; + + protected static final int PARALLELISM = 4; + + /** + * Test program with JobManager failure. + * + * @param zkQuorum ZooKeeper quorum to connect to + * @param coordinateDir Coordination directory + * @throws Exception + */ + public abstract void testJobManagerFailure(String zkQuorum, File coordinateDir) throws Exception; + + @Test + public void testJobManagerProcessFailure() throws Exception { + // Config + final int numberOfJobManagers = 2; + final int numberOfTaskManagers = 2; + final int numberOfSlotsPerTaskManager = 2; + + assertEquals(PARALLELISM, numberOfTaskManagers * numberOfSlotsPerTaskManager); + + // Setup + // Test actor system + ActorSystem testActorSystem; + + // Job managers + final JobManagerProcess[] jmProcess = new JobManagerProcess[numberOfJobManagers]; + + // Task managers + final ActorSystem[] tmActorSystem = new ActorSystem[numberOfTaskManagers]; + + // Leader election service + LeaderRetrievalService leaderRetrievalService = null; + + // Coordination between the processes goes through a directory + File coordinateTempDir = null; + + try { + final Deadline deadline = TestTimeOut.fromNow(); + + // Coordination directory + coordinateTempDir = createTempDirectory(); + + // Job Managers + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); + + // Start first process + jmProcess[0] = new JobManagerProcess(0, config); + jmProcess[0].createAndStart(); + + // Task manager configuration + config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 4); + config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 100); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); + + // Start the task manager process + for (int i = 0; i < numberOfTaskManagers; i++) { + tmActorSystem[i] = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); + TaskManager.startTaskManagerComponentsAndActor( + config, tmActorSystem[i], "localhost", + Option.empty(), Option.empty(), + false, StreamingMode.STREAMING, TaskManager.class); + } + + // Test actor system + testActorSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); + + jmProcess[0].getActorRef(testActorSystem, deadline.timeLeft()); + + // Leader listener + TestingListener leaderListener = new TestingListener(); + leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config); + leaderRetrievalService.start(leaderListener); + + // Initial submission + leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + + String leaderAddress = leaderListener.getAddress(); + UUID leaderId = leaderListener.getLeaderSessionID(); + + // Get the leader ref + ActorRef leaderRef = AkkaUtils.getActorRef(leaderAddress, testActorSystem, deadline.timeLeft()); + ActorGateway leaderGateway = new AkkaActorGateway(leaderRef, leaderId); + + // Wait for all task managers to connect to the leading job manager + JobManagerActorTestUtils.waitForTaskManagers(numberOfTaskManagers, leaderGateway, + deadline.timeLeft()); + + final File coordinateDirClosure = coordinateTempDir; + final Throwable[] errorRef = new Throwable[1]; + + // we trigger program execution in a separate thread + Thread programTrigger = new Thread("Program Trigger") { + @Override + public void run() { + try { + testJobManagerFailure(ZooKeeper.getConnectString(), coordinateDirClosure); + } + catch (Throwable t) { + t.printStackTrace(); + errorRef[0] = t; + } + } + }; + + //start the test program + programTrigger.start(); + + // wait until all marker files are in place, indicating that all tasks have started + AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(coordinateTempDir, + READY_MARKER_FILE_PREFIX, PARALLELISM, deadline.timeLeft().toMillis()); + + // Kill one of the job managers and trigger recovery + jmProcess[0].destroy(); + + jmProcess[1] = new JobManagerProcess(1, config); + jmProcess[1].createAndStart(); + + jmProcess[1].getActorRef(testActorSystem, deadline.timeLeft()); + + // we create the marker file which signals the program functions tasks that they can complete + AbstractTaskManagerProcessFailureRecoveryTest.touchFile(new File(coordinateTempDir, PROCEED_MARKER_FILE)); + + programTrigger.join(deadline.timeLeft().toMillis()); + + // We wait for the finish marker file. We don't wait for the program trigger, because + // we submit in detached mode. + AbstractTaskManagerProcessFailureRecoveryTest.waitForMarkerFiles(coordinateTempDir, + FINISH_MARKER_FILE_PREFIX, 1, deadline.timeLeft().toMillis()); + + // check that the program really finished + assertFalse("The program did not finish in time", programTrigger.isAlive()); + + // check whether the program encountered an error + if (errorRef[0] != null) { + Throwable error = errorRef[0]; + error.printStackTrace(); + fail("The program encountered a " + error.getClass().getSimpleName() + " : " + error.getMessage()); + } + } + catch (Exception e) { + e.printStackTrace(); + + for (JobManagerProcess p : jmProcess) { + if (p != null) { + p.printProcessLog(); + } + } + + fail(e.getMessage()); + } + finally { + for (int i = 0; i < numberOfTaskManagers; i++) { + if (tmActorSystem[i] != null) { + tmActorSystem[i].shutdown(); + } + } + + if (leaderRetrievalService != null) { + leaderRetrievalService.stop(); + } + + for (JobManagerProcess jmProces : jmProcess) { + if (jmProces != null) { + jmProces.destroy(); + } + } + + // Delete coordination directory + if (coordinateTempDir != null) { + try { + FileUtils.deleteDirectory(coordinateTempDir); + } + catch (Throwable ignored) { + } + } + } + } + +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java similarity index 87% rename from flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java rename to flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index 7e16baf0982d1..c02fa6cdf2dd4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -50,9 +50,7 @@ import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import java.io.InputStream; import java.io.StringWriter; -import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; @@ -75,10 +73,11 @@ * guaranteed to remain empty (all tasks are already deployed) and kills one of * the original task managers. The recovery should restart the tasks on the new TaskManager. */ -public abstract class AbstractProcessFailureRecoveryTest extends TestLogger { +public abstract class AbstractTaskManagerProcessFailureRecoveryTest extends TestLogger { protected static final String READY_MARKER_FILE_PREFIX = "ready_"; protected static final String PROCEED_MARKER_FILE = "proceed"; + protected static final String FINISH_MARKER_FILE_PREFIX = "finish_"; protected static final int PARALLELISM = 4; @@ -111,7 +110,7 @@ public void testTaskManagerProcessFailure() { CommonTestUtils.printLog4jDebugConfig(tempLogFile); // coordination between the processes goes through a directory - coordinateTempDir = createTempDirectory(); + coordinateTempDir = CommonTestUtils.createTempDirectory(); // find a free port to start the JobManager final int jobManagerPort = NetUtils.getAvailablePort(); @@ -146,9 +145,9 @@ public void testTaskManagerProcessFailure() { // start the first two TaskManager processes taskManagerProcess1 = new ProcessBuilder(command).start(); - new PipeForwarder(taskManagerProcess1.getErrorStream(), processOutput1); + new CommonTestUtils.PipeForwarder(taskManagerProcess1.getErrorStream(), processOutput1); taskManagerProcess2 = new ProcessBuilder(command).start(); - new PipeForwarder(taskManagerProcess2.getErrorStream(), processOutput2); + new CommonTestUtils.PipeForwarder(taskManagerProcess2.getErrorStream(), processOutput2); // we wait for the JobManager to have the two TaskManagers available // since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes) @@ -166,7 +165,7 @@ public void testTaskManagerProcessFailure() { @Override public void run() { try { - testProgram(jobManagerPort, coordinateDirClosure); + testTaskManagerFailure(jobManagerPort, coordinateDirClosure); } catch (Throwable t) { t.printStackTrace(); @@ -180,7 +179,7 @@ public void run() { // wait until all marker files are in place, indicating that all tasks have started // max 20 seconds - if (!waitForMarkerFiles(coordinateTempDir, PARALLELISM, 120000)) { + if (!waitForMarkerFiles(coordinateTempDir, READY_MARKER_FILE_PREFIX, PARALLELISM, 120000)) { // check if the program failed for some reason if (errorRef.get() != null) { Throwable error = errorRef.get(); @@ -195,7 +194,7 @@ public void run() { // start the third TaskManager taskManagerProcess3 = new ProcessBuilder(command).start(); - new PipeForwarder(taskManagerProcess3.getErrorStream(), processOutput3); + new CommonTestUtils.PipeForwarder(taskManagerProcess3.getErrorStream(), processOutput3); // we wait for the third TaskManager to register // since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes) @@ -269,7 +268,7 @@ public void run() { * @param coordinateDir TaskManager failure will be triggered only after processes * have successfully created file under this directory */ - public abstract void testProgram(int jobManagerPort, File coordinateDir) throws Exception; + public abstract void testTaskManagerFailure(int jobManagerPort, File coordinateDir) throws Exception; protected void waitUntilNumTaskManagersAreRegistered(ActorRef jobManager, int numExpected, long maxDelay) @@ -316,20 +315,6 @@ protected static void printProcessLog(String processName, String log) { System.out.println("-----------------------------------------"); } - protected static File createTempDirectory() throws IOException { - File tempDir = new File(System.getProperty("java.io.tmpdir")); - - for (int i = 0; i < 10; i++) { - File dir = new File(tempDir, UUID.randomUUID().toString()); - if (!dir.exists() && dir.mkdirs()) { - return dir; - } - System.err.println("Could not use temporary directory " + dir.getAbsolutePath()); - } - - throw new IOException("Could not create temporary file directory"); - } - protected static void touchFile(File file) throws IOException { if (!file.exists()) { new FileOutputStream(file).close(); @@ -339,7 +324,7 @@ protected static void touchFile(File file) throws IOException { } } - protected static boolean waitForMarkerFiles(File basedir, int num, long timeout) { + protected static boolean waitForMarkerFiles(File basedir, String prefix, int num, long timeout) { long now = System.currentTimeMillis(); final long deadline = now + timeout; @@ -348,7 +333,7 @@ protected static boolean waitForMarkerFiles(File basedir, int num, long timeout) boolean allFound = true; for (int i = 0; i < num; i++) { - File nextToCheck = new File(basedir, READY_MARKER_FILE_PREFIX + i); + File nextToCheck = new File(basedir, prefix + i); if (!nextToCheck.exists()) { allFound = false; break; @@ -409,36 +394,4 @@ public static void main(String[] args) { } } - /** - * Utility class to read the output of a process stream and forward it into a StringWriter. - */ - protected static class PipeForwarder extends Thread { - - private final StringWriter target; - private final InputStream source; - - public PipeForwarder(InputStream source, StringWriter target) { - super("Pipe Forwarder"); - setDaemon(true); - - this.source = source; - this.target = target; - - start(); - } - - @Override - public void run() { - try { - int next; - while ((next = source.read()) != -1) { - target.write(next); - } - } - catch (IOException e) { - // terminate - } - } - } - } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java new file mode 100644 index 0000000000000..a0c831238c474 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java @@ -0,0 +1,713 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.recovery; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.leaderelection.TestingListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; +import org.apache.flink.runtime.testutils.JobManagerProcess; +import org.apache.flink.runtime.testutils.TaskManagerProcess; +import org.apache.flink.runtime.testutils.TestJvmProcess; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.junit.AfterClass; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +@Ignore +public class ChaosMonkeyITCase { + + private static final Logger LOG = LoggerFactory.getLogger(ChaosMonkeyITCase.class); + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static File FileStateBackendBasePath; + + private final static File CheckpointCompletedCoordination; + + private final static File ProceedCoordination; + + private final static String COMPLETED_PREFIX = "completed_"; + + private final static long LastElement = -1; + + private final Random rand = new Random(); + + private int jobManagerPid; + private int taskManagerPid; + + static { + try { + FileStateBackendBasePath = CommonTestUtils.createTempDirectory(); + CheckpointCompletedCoordination = new File(FileStateBackendBasePath, COMPLETED_PREFIX); + ProceedCoordination = new File(FileStateBackendBasePath, "proceed"); + } + catch (IOException e) { + throw new RuntimeException("Error in test setup. Could not create directory.", e); + } + } + + @AfterClass + public static void tearDown() throws Exception { + if (ZooKeeper != null) { + ZooKeeper.shutdown(); + } + + if (FileStateBackendBasePath != null) { + FileUtils.deleteDirectory(FileStateBackendBasePath); + } + } + + @Test + public void testChaosMonkey() throws Exception { + // Test config + final int numberOfJobManagers = 3; + final int numberOfTaskManagers = 3; + final int numberOfSlotsPerTaskManager = 2; + + // The final count each source is counting to: 1...n + final int n = 5000; + + // Parallelism for the program + final int parallelism = numberOfTaskManagers * numberOfSlotsPerTaskManager; + + // The test should not run longer than this + final FiniteDuration testDuration = new FiniteDuration(10, TimeUnit.MINUTES); + + // Every x seconds a random job or task manager is killed + // + // The job will will be running for $killEvery seconds and then a random Job/TaskManager + // will be killed. On recovery (which takes some time to bring up the new process etc.), + // this test will wait for task managers to reconnect before starting the next count down. + // Therefore the delay between retries is not important in this setup. + final FiniteDuration killEvery = new FiniteDuration(30, TimeUnit.SECONDS); + + // Trigger a checkpoint every + final int checkpointingIntervalMs = 2000; + + // Total number of kills + final int totalNumberOfKills = 5; + + // ----------------------------------------------------------------------------------------- + + // Setup + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + ZooKeeper.getConnectString(), FileStateBackendBasePath.getPath()); + + // Akka and restart timeouts + config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, "1000 ms"); + config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "6 s"); + config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 9); + + if (checkpointingIntervalMs >= killEvery.toMillis()) { + throw new IllegalArgumentException("Relax! You want to kill processes every " + + killEvery + ", but the checkpointing interval is " + + checkpointingIntervalMs / 1000 + " seconds. Either decrease the interval or " + + "increase the kill interval. Otherwise, the program will not complete any " + + "checkpoint."); + } + + // Task manager + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numberOfSlotsPerTaskManager); + + ActorSystem testActorSystem = null; + LeaderRetrievalService leaderRetrievalService = null; + List jobManagerProcesses = new ArrayList<>(); + List taskManagerProcesses = new ArrayList<>(); + + try { + // Initial state + for (int i = 0; i < numberOfJobManagers; i++) { + jobManagerProcesses.add(createAndStartJobManagerProcess(config)); + } + + for (int i = 0; i < numberOfTaskManagers; i++) { + taskManagerProcesses.add(createAndStartTaskManagerProcess(config)); + } + + testActorSystem = AkkaUtils.createDefaultActorSystem(); + + // Leader listener + leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config); + TestingListener leaderListener = new TestingListener(); + leaderRetrievalService.start(leaderListener); + + Deadline deadline = testDuration.fromNow(); + + // Wait for the new leader + int leaderIndex = waitForNewLeader( + leaderListener, jobManagerProcesses, deadline.timeLeft()); + + // Wait for the task managers to connect + waitForTaskManagers( + numberOfTaskManagers, + jobManagerProcesses.get(leaderIndex), + testActorSystem, + deadline.timeLeft()); + + // The job + JobGraph jobGraph = createJobGraph(n, CheckpointCompletedCoordination.getPath(), + ProceedCoordination.getPath(), parallelism, checkpointingIntervalMs); + + LOG.info("Submitting job {}", jobGraph.getJobID()); + submitJobGraph(jobGraph, jobManagerProcesses.get(leaderIndex), leaderListener, + testActorSystem, deadline.timeLeft()); + + LOG.info("Waiting for a checkpoint to complete before kicking off chaos"); + + // Wait for a checkpoint to complete + TestJvmProcess.waitForMarkerFiles(FileStateBackendBasePath, COMPLETED_PREFIX, + parallelism, deadline.timeLeft().toMillis()); + + LOG.info("Checkpoint completed... ready for chaos"); + + int currentKillNumber = 1; + int currentJobManagerKills = 0; + int currentTaskManagerKills = 0; + + for (int i = 0; i < totalNumberOfKills; i++) { + LOG.info("Waiting for {} before next kill ({}/{})", killEvery, currentKillNumber++, totalNumberOfKills); + Thread.sleep(killEvery.toMillis()); + + LOG.info("Checking job status..."); + + JobStatus jobStatus = requestJobStatus(jobGraph.getJobID(), + jobManagerProcesses.get(leaderIndex), testActorSystem, deadline.timeLeft()); + + if (jobStatus != JobStatus.RUNNING && jobStatus != JobStatus.FINISHED) { + // Wait for it to run + LOG.info("Waiting for job status {}", JobStatus.RUNNING); + waitForJobRunning(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex), + testActorSystem, deadline.timeLeft()); + } + else if (jobStatus == JobStatus.FINISHED) { + // Early finish + LOG.info("Job finished"); + return; + } + else { + LOG.info("Job status is {}", jobStatus); + } + + if (rand.nextBoolean()) { + LOG.info("Killing the leading JobManager"); + + JobManagerProcess newJobManager = createAndStartJobManagerProcess(config); + + JobManagerProcess leader = jobManagerProcesses.remove(leaderIndex); + leader.destroy(); + currentJobManagerKills++; + + LOG.info("Killed {}", leader); + + // Make sure to add the new job manager before looking for a new leader + jobManagerProcesses.add(newJobManager); + + // Wait for the new leader + leaderIndex = waitForNewLeader( + leaderListener, jobManagerProcesses, deadline.timeLeft()); + + // Wait for the task managers to connect + waitForTaskManagers( + numberOfTaskManagers, + jobManagerProcesses.get(leaderIndex), + testActorSystem, + deadline.timeLeft()); + } + else { + LOG.info("Killing a random TaskManager"); + TaskManagerProcess newTaskManager = createAndStartTaskManagerProcess(config); + + // Wait for this new task manager to be connected + waitForTaskManagers( + numberOfTaskManagers + 1, + jobManagerProcesses.get(leaderIndex), + testActorSystem, + deadline.timeLeft()); + + // Now it's safe to kill a process + int next = rand.nextInt(numberOfTaskManagers); + TaskManagerProcess taskManager = taskManagerProcesses.remove(next); + + LOG.info("{} has been chosen. Killing process...", taskManager); + + taskManager.destroy(); + currentTaskManagerKills++; + + // Add the new task manager after killing an old one + taskManagerProcesses.add(newTaskManager); + } + } + + LOG.info("Chaos is over. Total kills: {} ({} job manager + {} task managers). " + + "Checking job status...", + totalNumberOfKills, currentJobManagerKills, currentTaskManagerKills); + + // Signal the job to speed up (if it is not done yet) + TestJvmProcess.touchFile(ProceedCoordination); + + // Wait for the job to finish + LOG.info("Waiting for job status {}", JobStatus.FINISHED); + waitForJobFinished(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex), + testActorSystem, deadline.timeLeft()); + + LOG.info("Job finished"); + + LOG.info("Waiting for job removal"); + waitForJobRemoved(jobGraph.getJobID(), jobManagerProcesses.get(leaderIndex), + testActorSystem, deadline.timeLeft()); + LOG.info("Job removed"); + + LOG.info("Checking clean recovery state..."); + checkCleanRecoveryState(config); + LOG.info("Recovery state clean"); + } + catch (Throwable t) { + System.out.println("#################################################"); + System.out.println(" TASK MANAGERS"); + System.out.println("#################################################"); + + for (TaskManagerProcess taskManagerProcess : taskManagerProcesses) { + taskManagerProcess.printProcessLog(); + } + + System.out.println("#################################################"); + System.out.println(" JOB MANAGERS"); + System.out.println("#################################################"); + + for (JobManagerProcess jobManagerProcess : jobManagerProcesses) { + jobManagerProcess.printProcessLog(); + } + + t.printStackTrace(); + } + finally { + for (JobManagerProcess jobManagerProcess : jobManagerProcesses) { + if (jobManagerProcess != null) { + jobManagerProcess.destroy(); + } + } + + if (leaderRetrievalService != null) { + leaderRetrievalService.stop(); + } + + if (testActorSystem != null) { + testActorSystem.shutdown(); + } + } + } + + // - The test program -------------------------------------------------------------------------- + + private JobGraph createJobGraph( + int n, + String completedCheckpointMarker, + String proceedMarker, + int parallelism, + int checkpointingIntervalMs) { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(parallelism); + env.enableCheckpointing(checkpointingIntervalMs); + + int expectedResult = parallelism * n * (n + 1) / 2; + + env.addSource(new CheckpointedSequenceSource(n, completedCheckpointMarker, proceedMarker)) + .addSink(new CountingSink(parallelism, expectedResult)) + .setParallelism(1); + + return env.getStreamGraph().getJobGraph(); + } + + public static class CheckpointedSequenceSource extends RichParallelSourceFunction + implements Checkpointed, CheckpointNotifier { + + private static final long serialVersionUID = 0L; + + private final long end; + + private final String completedCheckpointMarkerFilePath; + + private final File proceedFile; + + private long current = 0; + + private volatile boolean isRunning = true; + + public CheckpointedSequenceSource(long end, String completedCheckpointMarkerFilePath, String proceedMarkerFilePath) { + checkArgument(end >= 0, "Negative final count"); + + this.end = end; + this.completedCheckpointMarkerFilePath = completedCheckpointMarkerFilePath; + this.proceedFile = new File(proceedMarkerFilePath); + } + + @Override + public void run(SourceContext ctx) throws Exception { + while (isRunning) { + + if (!proceedFile.exists()) { + Thread.sleep(50); + } + + synchronized (ctx.getCheckpointLock()) { + if (current <= end) { + ctx.collect(current++); + } + else { + ctx.collect(LastElement); + return; + } + } + } + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + LOG.info("Snapshotting state {} @ ID {}.", current, checkpointId); + return current; + } + + @Override + public void restoreState(Long state) { + LOG.info("Restoring state {}/{}", state, end); + current = state; + } + + + @Override + public void cancel() { + isRunning = false; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.info("Checkpoint {} completed.", checkpointId); + + int taskIndex = getRuntimeContext().getIndexOfThisSubtask(); + TestJvmProcess.touchFile(new File(completedCheckpointMarkerFilePath + taskIndex)); + } + } + + public static class CountingSink extends RichSinkFunction + implements Checkpointed, CheckpointNotifier { + + private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class); + + private static final long serialVersionUID = 0L; + + private final int parallelism; + + private final long expectedFinalCount; + + private long current; + + private int numberOfReceivedLastElements; + + + public CountingSink(int parallelism, long expectedFinalCount) { + this.expectedFinalCount = expectedFinalCount; + this.parallelism = parallelism; + } + + @Override + public void invoke(Long value) throws Exception { + if (value == LastElement) { + numberOfReceivedLastElements++; + + if (numberOfReceivedLastElements == parallelism) { + if (current != expectedFinalCount) { + throw new Exception("Unexpected final result " + current); + } + else { + LOG.info("Final result " + current); + } + } + else if (numberOfReceivedLastElements > parallelism) { + throw new IllegalStateException("Received more elements than parallelism."); + } + } + else { + current += value; + } + } + + @Override + public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + LOG.info("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId); + return this; + } + + @Override + public void restoreState(CountingSink state) { + LOG.info("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements); + this.current = state.current; + this.numberOfReceivedLastElements = state.numberOfReceivedLastElements; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.info("Checkpoint {} completed.", checkpointId); + } + } + + // - Utilities --------------------------------------------------------------------------------- + + private void submitJobGraph( + JobGraph jobGraph, + JobManagerProcess jobManager, + TestingListener leaderListener, + ActorSystem actorSystem, + FiniteDuration timeout) throws Exception { + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + UUID jobManagerLeaderId = leaderListener.getLeaderSessionID(); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, jobManagerLeaderId); + + jobManagerGateway.tell(new JobManagerMessages.SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); + } + + private void checkCleanRecoveryState(Configuration config) throws Exception { + LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); + List jobGraphs = ZooKeeper.getClient().getChildren() + .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); + assertEquals("Unclean job graphs: " + jobGraphs, 0, jobGraphs.size()); + + LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); + List checkpoints = ZooKeeper.getClient().getChildren() + .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); + assertEquals("Unclean checkpoints: " + checkpoints, 0, checkpoints.size()); + + LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); + List checkpointCounter = ZooKeeper.getClient().getChildren() + .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); + assertEquals("Unclean checkpoint counter: " + checkpointCounter, 0, checkpointCounter.size()); + + LOG.info("ZooKeeper state is clean"); + + LOG.info("Checking file system backend state..."); + + File fsCheckpoints = new File(config.getString(ConfigConstants.STATE_BACKEND_FS_DIR, "")); + + LOG.info("Checking " + fsCheckpoints); + + File[] files = fsCheckpoints.listFiles(); + if (files == null) { + fail(fsCheckpoints + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles())); + } + else { + assertEquals("Unclean file system checkpoints: " + Arrays.toString(fsCheckpoints.listFiles()), + 0, files.length); + } + + File fsRecovery = new File(config.getString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, "")); + + LOG.info("Checking " + fsRecovery); + + files = fsRecovery.listFiles(); + if (files == null) { + fail(fsRecovery + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles())); + } + else { + assertEquals("Unclean file system checkpoints: " + Arrays.toString(fsRecovery.listFiles()), + 0, files.length); + } + } + + private void waitForJobRemoved( + JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout) + throws Exception { + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null); + + Future archiveFuture = jobManagerGateway.ask(JobManagerMessages.getRequestArchive(), timeout); + + ActorRef archive = ((JobManagerMessages.ResponseArchive) Await.result(archiveFuture, timeout)).actor(); + + AkkaActorGateway archiveGateway = new AkkaActorGateway(archive, null); + + Deadline deadline = timeout.fromNow(); + + while (deadline.hasTimeLeft()) { + JobManagerMessages.JobStatusResponse resp = JobManagerActorTestUtils + .requestJobStatus(jobId, archiveGateway, deadline.timeLeft()); + + if (resp instanceof JobManagerMessages.JobNotFound) { + Thread.sleep(100); + } + else { + return; + } + } + } + + private JobStatus requestJobStatus( + JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout) + throws Exception { + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null); + + JobManagerMessages.JobStatusResponse resp = JobManagerActorTestUtils + .requestJobStatus(jobId, jobManagerGateway, timeout); + + if (resp instanceof JobManagerMessages.CurrentJobStatus) { + JobManagerMessages.CurrentJobStatus jobStatusResponse = (JobManagerMessages + .CurrentJobStatus) resp; + + return jobStatusResponse.status(); + } + else if (resp instanceof JobManagerMessages.JobNotFound) { + return JobStatus.RESTARTING; + } + + throw new IllegalStateException("Unexpected response from JobManager"); + } + + private void waitForJobRunning( + JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout) + throws Exception { + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null); + + JobManagerActorTestUtils.waitForJobStatus(jobId, JobStatus.RUNNING, jobManagerGateway, timeout); + } + + private void waitForJobFinished( + JobID jobId, JobManagerProcess jobManager, ActorSystem actorSystem, FiniteDuration timeout) + throws Exception { + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null); + + JobManagerActorTestUtils.waitForJobStatus(jobId, JobStatus.FINISHED, jobManagerGateway, timeout); + } + + private void waitForTaskManagers( + int minimumNumberOfTaskManagers, + JobManagerProcess jobManager, + ActorSystem actorSystem, + FiniteDuration timeout) throws Exception { + + LOG.info("Waiting for {} task managers to connect to leading {}", + minimumNumberOfTaskManagers, jobManager); + + ActorRef jobManagerRef = jobManager.getActorRef(actorSystem, timeout); + AkkaActorGateway jobManagerGateway = new AkkaActorGateway(jobManagerRef, null); + + JobManagerActorTestUtils.waitForTaskManagers( + minimumNumberOfTaskManagers, jobManagerGateway, timeout); + + LOG.info("All task managers connected"); + } + + private int waitForNewLeader( + TestingListener leaderListener, + List jobManagerProcesses, + FiniteDuration timeout) throws Exception { + + LOG.info("Waiting for new leader notification"); + leaderListener.waitForNewLeader(timeout.toMillis()); + + LOG.info("Leader: {}:{}", leaderListener.getAddress(), leaderListener.getLeaderSessionID()); + + String currentLeader = leaderListener.getAddress(); + + int leaderIndex = -1; + + for (int i = 0; i < jobManagerProcesses.size(); i++) { + JobManagerProcess jobManager = jobManagerProcesses.get(i); + if (jobManager.getJobManagerAkkaURL().equals(currentLeader)) { + leaderIndex = i; + break; + } + } + + if (leaderIndex == -1) { + throw new IllegalStateException("Failed to determine which process is leader"); + } + + return leaderIndex; + } + + private JobManagerProcess createAndStartJobManagerProcess(Configuration config) + throws Exception { + + JobManagerProcess jobManager = new JobManagerProcess(jobManagerPid++, config); + jobManager.createAndStart(); + LOG.info("Created and started {}.", jobManager); + + return jobManager; + } + + private TaskManagerProcess createAndStartTaskManagerProcess(Configuration config) + throws Exception { + + TaskManagerProcess taskManager = new TaskManagerProcess(taskManagerPid++, config); + taskManager.createAndStart(); + LOG.info("Created and started {}.", taskManager); + + return taskManager; + } + +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java new file mode 100644 index 0000000000000..54ddf7e300b1c --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerCheckpointRecoveryITCase.java @@ -0,0 +1,395 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.recovery; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import org.apache.commons.io.FileUtils; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.StreamingMode; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.leaderelection.TestingListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.JobManagerActorTestUtils; +import org.apache.flink.runtime.testutils.JobManagerProcess; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; +import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLongArray; +import java.util.concurrent.atomic.AtomicReference; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.flink.runtime.messages.JobManagerMessages.SubmitJob; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class JobManagerCheckpointRecoveryITCase { + + private final static ZooKeeperTestEnvironment ZooKeeper = new ZooKeeperTestEnvironment(1); + + private final static FiniteDuration TestTimeOut = new FiniteDuration(5, TimeUnit.MINUTES); + + private static final File FileStateBackendBasePath; + + static { + try { + FileStateBackendBasePath = CommonTestUtils.createTempDirectory(); + } + catch (IOException e) { + throw new RuntimeException("Error in test setup. Could not create directory.", e); + } + } + + @AfterClass + public static void tearDown() throws Exception { + ZooKeeper.shutdown(); + + if (FileStateBackendBasePath != null) { + FileUtils.deleteDirectory(FileStateBackendBasePath); + } + } + + @Before + public void cleanUp() throws Exception { + if (FileStateBackendBasePath != null) { + FileUtils.cleanDirectory(FileStateBackendBasePath); + } + + ZooKeeper.deleteAll(); + } + + // --------------------------------------------------------------------------------------------- + + private static final int Parallelism = 8; + + private static final CountDownLatch CompletedCheckpointsLatch = new CountDownLatch(2); + + private static final AtomicLongArray RecoveredStates = new AtomicLongArray(Parallelism); + + private static final CountDownLatch FinalCountLatch = new CountDownLatch(1); + + private static final AtomicReference FinalCount = new AtomicReference<>(); + + private static final long LastElement = -1; + + /** + * Simple checkpointed streaming sum. + * + *

The sources (Parallelism) count until sequenceEnd. The sink (1) sums up all counts and + * returns it to the main thread via a static variable. We wait until some checkpoints are + * completed and sanity check that the sources recover with an updated state to make sure that + * this test actually tests something. + */ + @Test + public void testCheckpointedStreamingSumProgram() throws Exception { + // Config + final int checkpointingInterval = 200; + final int sequenceEnd = 5000; + final long expectedSum = Parallelism * sequenceEnd * (sequenceEnd + 1) / 2; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); + env.setParallelism(Parallelism); + env.enableCheckpointing(checkpointingInterval); + + env + .addSource(new CheckpointedSequenceSource(sequenceEnd)) + .addSink(new CountingSink()) + .setParallelism(1); + + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig(ZooKeeper + .getConnectString(), FileStateBackendBasePath.getPath()); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, Parallelism); + + ActorSystem testSystem = null; + JobManagerProcess[] jobManagerProcess = new JobManagerProcess[2]; + LeaderRetrievalService leaderRetrievalService = null; + ActorSystem taskManagerSystem = null; + + try { + final Deadline deadline = TestTimeOut.fromNow(); + + // Test actor system + testSystem = AkkaUtils.createActorSystem(new Configuration(), + new Some<>(new Tuple2("localhost", 0))); + + // The job managers + jobManagerProcess[0] = new JobManagerProcess(0, config); + jobManagerProcess[1] = new JobManagerProcess(1, config); + + jobManagerProcess[0].createAndStart(); + jobManagerProcess[1].createAndStart(); + + // Leader listener + TestingListener leaderListener = new TestingListener(); + leaderRetrievalService = ZooKeeperUtils.createLeaderRetrievalService(config); + leaderRetrievalService.start(leaderListener); + + // The task manager + taskManagerSystem = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig()); + TaskManager.startTaskManagerComponentsAndActor( + config, taskManagerSystem, "localhost", + Option.empty(), Option.empty(), + false, StreamingMode.STREAMING, TaskManager.class); + + { + // Initial submission + leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + + String leaderAddress = leaderListener.getAddress(); + UUID leaderId = leaderListener.getLeaderSessionID(); + + // Get the leader ref + ActorRef leaderRef = AkkaUtils.getActorRef( + leaderAddress, testSystem, deadline.timeLeft()); + ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId); + + // Submit the job in detached mode + leader.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED)); + + JobManagerActorTestUtils.waitForJobStatus( + jobGraph.getJobID(), JobStatus.RUNNING, leader, deadline.timeLeft()); + } + + // Who's the boss? + JobManagerProcess leadingJobManagerProcess; + if (jobManagerProcess[0].getJobManagerAkkaURL().equals(leaderListener.getAddress())) { + leadingJobManagerProcess = jobManagerProcess[0]; + } + else { + leadingJobManagerProcess = jobManagerProcess[1]; + } + + CompletedCheckpointsLatch.await(); + + // Kill the leading job manager process + leadingJobManagerProcess.destroy(); + + { + // Recovery by the standby JobManager + leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + + String leaderAddress = leaderListener.getAddress(); + UUID leaderId = leaderListener.getLeaderSessionID(); + + ActorRef leaderRef = AkkaUtils.getActorRef( + leaderAddress, testSystem, deadline.timeLeft()); + ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId); + + JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING, + leader, deadline.timeLeft()); + } + + // Wait to finish + FinalCountLatch.await(); + + assertEquals(expectedSum, (long) FinalCount.get()); + + for (int i = 0; i < Parallelism; i++) { + assertNotEquals(0, RecoveredStates.get(i)); + } + } + catch (Throwable t) { + // In case of an error, print the job manager process logs. + if (jobManagerProcess[0] != null) { + jobManagerProcess[0].printProcessLog(); + } + + if (jobManagerProcess[1] != null) { + jobManagerProcess[1].printProcessLog(); + } + + t.printStackTrace(); + } + finally { + if (jobManagerProcess[0] != null) { + jobManagerProcess[0].destroy(); + } + + if (jobManagerProcess[1] != null) { + jobManagerProcess[1].destroy(); + } + + if (leaderRetrievalService != null) { + leaderRetrievalService.stop(); + } + + if (taskManagerSystem != null) { + taskManagerSystem.shutdown(); + } + + if (testSystem != null) { + testSystem.shutdown(); + } + } + } + + // --------------------------------------------------------------------------------------------- + + /** + * A checkpointed source, which emits elements from 0 to a configured number. + */ + public static class CheckpointedSequenceSource extends RichParallelSourceFunction + implements Checkpointed { + + private static final Logger LOG = LoggerFactory.getLogger(CheckpointedSequenceSource.class); + + private static final long serialVersionUID = 0L; + + private static final CountDownLatch sync = new CountDownLatch(Parallelism); + + private final long end; + + private long current = 0; + + private volatile boolean isRunning = true; + + public CheckpointedSequenceSource(long end) { + checkArgument(end >= 0, "Negative final count"); + this.end = end; + } + + @Override + public void run(SourceContext ctx) throws Exception { + while (isRunning) { + synchronized (ctx.getCheckpointLock()) { + if (current <= end) { + ctx.collect(current++); + } + else { + ctx.collect(LastElement); + return; + } + } + + // Slow down until some checkpoints are completed + if (sync.getCount() != 0) { + Thread.sleep(100); + } + } + } + + @Override + public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + LOG.debug("Snapshotting state {} @ ID {}.", current, checkpointId); + return current; + } + + @Override + public void restoreState(Long state) { + LOG.debug("Restoring state {}", state); + + // This is necessary to make sure that something is recovered at all. Otherwise it + // might happen that the job is restarted from the beginning. + RecoveredStates.set(getRuntimeContext().getIndexOfThisSubtask(), state); + + sync.countDown(); + + current = state; + } + + @Override + public void cancel() { + isRunning = false; + } + } + + /** + * A checkpointed sink, which sums up its input and notifies the main thread after all inputs + * are exhausted. + */ + public static class CountingSink implements SinkFunction, Checkpointed, + CheckpointNotifier { + + private static final Logger LOG = LoggerFactory.getLogger(CountingSink.class); + + private static final long serialVersionUID = 1436484290453629091L; + + private long current = 0; + + private int numberOfReceivedLastElements; + + @Override + public void invoke(Long value) throws Exception { + if (value == LastElement) { + numberOfReceivedLastElements++; + + if (numberOfReceivedLastElements == Parallelism) { + FinalCount.set(current); + FinalCountLatch.countDown(); + } + else if (numberOfReceivedLastElements > Parallelism) { + throw new IllegalStateException("Received more elements than parallelism."); + } + } + else { + current += value; + } + } + + @Override + public CountingSink snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + LOG.debug("Snapshotting state {}:{} @ ID {}.", current, numberOfReceivedLastElements, checkpointId); + return this; + } + + @Override + public void restoreState(CountingSink state) { + LOG.debug("Restoring state {}:{}", state.current, state.numberOfReceivedLastElements); + this.current = state.current; + this.numberOfReceivedLastElements = state.numberOfReceivedLastElements; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Checkpoint {} completed.", checkpointId); + CompletedCheckpointsLatch.countDown(); + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java new file mode 100644 index 0000000000000..66565dd2c62cf --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerProcessFailureBatchRecoveryITCase.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.recovery; + +import org.apache.flink.api.common.ExecutionMode; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Collector; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.util.Arrays; +import java.util.Collection; + +import static org.junit.Assert.assertEquals; + +/** + * Test the recovery of a simple batch program in the case of JobManager process failure. + */ +@SuppressWarnings("serial") +@RunWith(Parameterized.class) +public class JobManagerProcessFailureBatchRecoveryITCase extends AbstractJobManagerProcessFailureRecoveryITCase { + + // -------------------------------------------------------------------------------------------- + // Parametrization (run pipelined and batch) + // -------------------------------------------------------------------------------------------- + + private final ExecutionMode executionMode; + + public JobManagerProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) { + this.executionMode = executionMode; + } + + @Parameterized.Parameters + public static Collection executionMode() { + return Arrays.asList(new Object[][]{ + {ExecutionMode.PIPELINED}, + {ExecutionMode.BATCH}}); + } + + // -------------------------------------------------------------------------------------------- + // Test the program + // -------------------------------------------------------------------------------------------- + + // This is slightly modified copy the task manager process failure program. + @Override + public void testJobManagerFailure(String zkQuorum, final File coordinateDir) throws Exception { + Configuration config = new Configuration(); + config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER"); + config.setString(ConfigConstants.ZOOKEEPER_QUORUM_KEY, zkQuorum); + + ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment( + "leader", 1, config); + env.setParallelism(PARALLELISM); + env.setNumberOfExecutionRetries(1); + env.getConfig().setExecutionMode(executionMode); + env.getConfig().disableSysoutLogging(); + + final long NUM_ELEMENTS = 100000L; + final DataSet result = env.generateSequence(1, NUM_ELEMENTS) + // make sure every mapper is involved (no one is skipped because of lazy split assignment) + .rebalance() + // the majority of the behavior is in the MapFunction + .map(new RichMapFunction() { + + private final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE); + + private boolean markerCreated = false; + private boolean checkForProceedFile = true; + + @Override + public Long map(Long value) throws Exception { + if (!markerCreated) { + int taskIndex = getRuntimeContext().getIndexOfThisSubtask(); + AbstractTaskManagerProcessFailureRecoveryTest.touchFile( + new File(coordinateDir, READY_MARKER_FILE_PREFIX + taskIndex)); + markerCreated = true; + } + + // check if the proceed file exists + if (checkForProceedFile) { + if (proceedFile.exists()) { + checkForProceedFile = false; + } + else { + // otherwise wait so that we make slow progress + Thread.sleep(100); + } + } + return value; + } + }) + .reduce(new ReduceFunction() { + @Override + public Long reduce(Long value1, Long value2) { + return value1 + value2; + } + }) + // The check is done in the mapper, because the client can currently not handle + // job manager losses/reconnects. + .flatMap(new RichFlatMapFunction() { + @Override + public void flatMap(Long value, Collector out) throws Exception { + assertEquals(NUM_ELEMENTS * (NUM_ELEMENTS + 1L) / 2L, (long) value); + + int taskIndex = getRuntimeContext().getIndexOfThisSubtask(); + AbstractTaskManagerProcessFailureRecoveryTest.touchFile( + new File(coordinateDir, FINISH_MARKER_FILE_PREFIX + taskIndex)); + } + }); + + result.output(new DiscardingOutputFormat()); + + env.execute(); + } + +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index 945a78c44797a..6dce370ac73e3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -115,13 +115,13 @@ public void testCancelingOnProcessFailure() { "-Dlog4j.configuration=file:" + tempLogFile.getAbsolutePath(), "-Xms80m", "-Xmx80m", "-classpath", getCurrentClasspath(), - AbstractProcessFailureRecoveryTest.TaskManagerProcessEntryPoint.class.getName(), + AbstractTaskManagerProcessFailureRecoveryTest.TaskManagerProcessEntryPoint.class.getName(), String.valueOf(jobManagerPort) }; // start the first two TaskManager processes taskManagerProcess = new ProcessBuilder(command).start(); - new AbstractProcessFailureRecoveryTest.PipeForwarder(taskManagerProcess.getErrorStream(), processOutput); + new CommonTestUtils.PipeForwarder(taskManagerProcess.getErrorStream(), processOutput); // we wait for the JobManager to have the two TaskManagers available // since some of the CI environments are very hostile, we need to give this a lot of time (2 minutes) diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java similarity index 92% rename from flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java index f2b8c3197a11c..173c8ea0c7fd4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java @@ -37,7 +37,7 @@ */ @SuppressWarnings("serial") @RunWith(Parameterized.class) -public class ProcessFailureBatchRecoveryITCase extends AbstractProcessFailureRecoveryTest { +public class TaskManagerProcessFailureBatchRecoveryITCase extends AbstractTaskManagerProcessFailureRecoveryTest { // -------------------------------------------------------------------------------------------- // Parametrization (run pipelined and batch) @@ -45,7 +45,7 @@ public class ProcessFailureBatchRecoveryITCase extends AbstractProcessFailureRec private final ExecutionMode executionMode; - public ProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) { + public TaskManagerProcessFailureBatchRecoveryITCase(ExecutionMode executionMode) { this.executionMode = executionMode; } @@ -61,7 +61,7 @@ public static Collection executionMode() { // -------------------------------------------------------------------------------------------- @Override - public void testProgram(int jobManagerPort, final File coordinateDir) throws Exception { + public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) throws Exception { ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); env.setParallelism(PARALLELISM); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java similarity index 94% rename from flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java rename to flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java index e7b1668933587..9fb577b18924c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java @@ -42,7 +42,7 @@ /** * Test for streaming program behaviour in case of TaskManager failure - * based on {@link AbstractProcessFailureRecoveryTest}. + * based on {@link AbstractTaskManagerProcessFailureRecoveryTest}. * * The logic in this test is as follows: * - The source slowly emits records (every 10 msecs) until the test driver @@ -54,20 +54,20 @@ * TaskManager. */ @SuppressWarnings("serial") -public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailureRecoveryTest { +public class TaskManagerProcessFailureStreamingRecoveryITCase extends AbstractTaskManagerProcessFailureRecoveryTest { private static final int DATA_COUNT = 10000; @Override - public void testProgram(int jobManagerPort, final File coordinateDir) throws Exception { - + public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) throws Exception { + final File tempCheckpointDir = new File(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH), UUID.randomUUID().toString()); assertTrue("Cannot create directory for checkpoints", tempCheckpointDir.mkdirs()); StreamExecutionEnvironment env = StreamExecutionEnvironment - .createRemoteEnvironment("localhost", jobManagerPort); + .createRemoteEnvironment("localhost", jobManagerPort); env.setParallelism(PARALLELISM); env.getConfig().disableSysoutLogging(); env.setNumberOfExecutionRetries(1); @@ -82,7 +82,7 @@ public Long map(Long value) throws Exception { return value; } }).startNewChain() - // populate the coordinate directory so we can proceed to TaskManager failure + // populate the coordinate directory so we can proceed to TaskManager failure .map(new Mapper(coordinateDir)); //write result to temporary file @@ -112,7 +112,7 @@ public static class SleepyDurableGenerateSequence extends RichParallelSourceFunc private final long end; private volatile boolean isRunning = true; - + private OperatorState collected; public SleepyDurableGenerateSequence(File coordinateDir, long end) { @@ -151,7 +151,7 @@ public void run(SourceContext sourceCtx) throws Exception { } } } - + @Override public void open(Configuration conf) throws IOException { collected = getRuntimeContext().getOperatorState("count", 0L, false); @@ -162,7 +162,7 @@ public void cancel() { isRunning = false; } } - + public static class Mapper extends RichMapFunction { private boolean markerCreated = false; private File coordinateDir; diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 6035c4593fad2..ed2113a1b781c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -21,6 +21,7 @@ import akka.actor.ActorSystem; import akka.actor.Kill; import akka.actor.PoisonPill; +import org.apache.commons.io.FileUtils; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobClient; @@ -39,17 +40,41 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.test.util.ForkableFlinkMiniCluster; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.Test; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; -import static org.junit.Assert.*; +import java.io.File; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; public class ZooKeeperLeaderElectionITCase extends TestLogger { private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION(); + private static final File tempDirectory; + + static { + try { + tempDirectory = org.apache.flink.runtime.testutils + .CommonTestUtils.createTempDirectory(); + } + catch (IOException e) { + throw new RuntimeException("Test setup failed", e); + } + } + + @AfterClass + public static void tearDown() throws Exception { + if (tempDirectory != null) { + FileUtils.deleteDirectory(tempDirectory); + } + } + /** * Tests that the TaskManagers successfully register at the new leader once the old leader * is terminated. @@ -64,13 +89,15 @@ public void testTaskManagerRegistrationAtReelectedLeader() throws Exception { configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper"); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs); + configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem"); + configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath()); ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration); try { cluster.start(); - for(int i = 0; i < numJMs; i++) { + for (int i = 0; i < numJMs; i++) { ActorGateway leadingJM = cluster.getLeaderGateway(timeout); cluster.waitForTaskManagersToBeRegisteredAtJobManager(leadingJM.actor()); @@ -86,7 +113,8 @@ public void testTaskManagerRegistrationAtReelectedLeader() throws Exception { cluster.clearLeader(); leadingJM.tell(PoisonPill.getInstance()); } - } finally { + } + finally { cluster.stop(); } } @@ -110,6 +138,13 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM); + configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem"); + configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath()); + + // @TODO @tillrohrmann temporary "disable" recovery, because currently the client does + // not need to resubmit a failed job to a new leader. Should we keep this test and + // disable recovery fully or will this be subsumed by the real client changes anyways? + configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, timeout.toString()); Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true); @@ -152,7 +187,7 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { thread.start(); // Kill all JobManager except for two - for(int i = 0; i < numJMs - 2; i++) { + for (int i = 0; i < numJMs - 2; i++) { ActorGateway jm = cluster.getLeaderGateway(timeout); cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor()); @@ -184,17 +219,18 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { thread.join(timeout.toMillis()); - if(thread.isAlive()) { + if (thread.isAlive()) { jobSubmission.finished = true; fail("The job submission thread did not stop (meaning it did not succeeded in" + "executing the test job."); } - } finally { + } + finally { if (clientActorSystem != null) { cluster.shutdownJobClientActorSystem(clientActorSystem); } - if(thread != null && thread.isAlive() && jobSubmission != null) { + if (thread != null && thread.isAlive() && jobSubmission != null) { jobSubmission.finished = true; } cluster.stop(); @@ -219,7 +255,7 @@ public JobSubmitterRunnable( @Override public void run() { - while(!finished) { + while (!finished) { try { LeaderRetrievalService lrService = LeaderRetrievalUtils.createLeaderRetrievalService( @@ -240,11 +276,14 @@ public void run() { getClass().getClassLoader()); finished = true; - } catch (JobExecutionException e) { + } + catch (JobExecutionException e) { // This was expected, so just try again to submit the job - } catch (LeaderRetrievalException e) { + } + catch (LeaderRetrievalException e) { // This can also happen, so just try again to submit the job - } catch (Exception e) { + } + catch (Exception e) { // This was not expected... fail the test case e.printStackTrace(); fail("Caught unexpected exception in job submission test case."); diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala index 61eb6a57f952b..4ada21ed48335 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala @@ -29,8 +29,9 @@ import grizzled.slf4j.Logger import org.apache.flink.api.common.JobID import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants} import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.jobgraph.JobStatus -import org.apache.flink.runtime.jobmanager.JobManager +import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager} import org.apache.flink.runtime.leaderelection.LeaderElectionService import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatus, CurrentJobStatus, JobNotFound} @@ -88,7 +89,9 @@ class YarnJobManager( delayBetweenRetries: Long, timeout: FiniteDuration, mode: StreamingMode, - leaderElectionService: LeaderElectionService) + leaderElectionService: LeaderElectionService, + submittedJobGraphs : SubmittedJobGraphStore, + checkpointRecoveryFactory : CheckpointRecoveryFactory) extends JobManager( flinkConfiguration, executionContext, @@ -100,7 +103,9 @@ class YarnJobManager( delayBetweenRetries, timeout, mode, - leaderElectionService) { + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) { import context._ import scala.collection.JavaConverters._ From 670be21c02539855114e2cc836e5f49a72e5b4ab Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 9 Oct 2015 00:50:07 +0200 Subject: [PATCH 5/7] [FLINK-2354] [runtime] Remove state changing futures in JobManager Internal actor states must only be modified within the actor thread. This avoids all the well-known issues coming with concurrency. Fix RemoveCachedJob by introducing RemoveJob Fix JobManagerITCase Add removeJob which maintains the job in the SubmittedJobGraphStore Make revokeLeadership not remove the jobs from the state backend Fix shading problem with curator by hiding CuratorFramework in ChaosMonkeyITCase --- flink-runtime/pom.xml | 4 +- .../flink/runtime/jobmanager/JobManager.scala | 262 ++++++++++-------- .../runtime/messages/JobManagerMessages.scala | 17 ++ ...nagerSubmittedJobGraphsRecoveryITCase.java | 5 +- .../zookeeper/ZooKeeperTestEnvironment.java | 10 + .../src/test/resources/log4j-test.properties | 2 +- .../runtime/jobmanager/JobManagerITCase.scala | 2 +- .../flink-shaded-curator-recipes/pom.xml | 78 ++++++ .../flink-shaded-curator-test/pom.xml | 86 ++++++ flink-shaded-curator/pom.xml | 82 ++---- flink-tests/pom.xml | 7 + .../test/recovery/ChaosMonkeyITCase.java | 15 +- .../yarn/YARNHighAvailabilityITCase.java | 13 +- .../flink/yarn/TestingYarnJobManager.scala | 10 +- 14 files changed, 404 insertions(+), 189 deletions(-) create mode 100644 flink-shaded-curator/flink-shaded-curator-recipes/pom.xml create mode 100644 flink-shaded-curator/flink-shaded-curator-test/pom.xml diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 1802709b29a46..1f43a4164c6ba 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -200,7 +200,7 @@ under the License. org.apache.flink - flink-shaded-curator + flink-shaded-curator-recipes ${project.version} @@ -435,7 +435,7 @@ under the License. - org.apache.flink:flink-shaded-curator + org.apache.flink:flink-shaded-curator-recipes diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index a92fdcbe0b6bc..d2fe7f52981de 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -65,7 +65,6 @@ import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMess import org.apache.flink.util.{ExceptionUtils, InstantiationUtil, NetUtils} import scala.collection.JavaConverters._ -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ import scala.concurrent.duration._ import scala.concurrent.forkjoin.ForkJoinPool @@ -128,6 +127,9 @@ class JobManager( var leaderSessionID: Option[UUID] = None + /** Futures which have to be completed before terminating the job manager */ + var futuresToComplete: Option[Seq[Future[Unit]]] = None + /** * Run when the job manager is started. Simply logs an informational message. * The method also starts the leader election service. @@ -164,7 +166,16 @@ class JobManager( override def postStop(): Unit = { log.info(s"Stopping JobManager ${getAddress}.") - cancelAndClearEverything(new Exception("The JobManager is shutting down.")) + val newFuturesToComplete = cancelAndClearEverything( + new Exception("The JobManager is shutting down."), + true) + + implicit val executionContext = context.dispatcher + + val futureToComplete = Future.sequence( + futuresToComplete.getOrElse(Seq()) ++ newFuturesToComplete) + + Await.ready(futureToComplete, timeout) // disconnect the registered task managers instanceManager.getAllRegisteredInstances.asScala.foreach { @@ -236,9 +247,11 @@ class JobManager( case RevokeLeadership => log.info(s"JobManager ${self.path.toSerializationFormat} was revoked leadership.") - future { - cancelAndClearEverything(new Exception("JobManager is no longer the leader.")) - }(context.dispatcher) + val newFuturesToComplete = cancelAndClearEverything( + new Exception("JobManager is no longer the leader."), + false) + + futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) ++ newFuturesToComplete) // disconnect the registered task managers instanceManager.getAllRegisteredInstances.asScala.foreach { @@ -316,9 +329,15 @@ class JobManager( val jobInfo = new JobInfo(client, listeningBehaviour, System.currentTimeMillis(), jobGraph.getSessionTimeout) - future { - submitJob(jobGraph, jobInfo) - }(context.dispatcher) + submitJob(jobGraph, jobInfo) + + case RecoverSubmittedJob(submittedJobGraph) => + if (!currentJobs.contains(submittedJobGraph.getJobId)) { + submitJob( + submittedJobGraph.getJobGraph(), + submittedJobGraph.getJobInfo(), + isRecovery = true) + } case RecoverJob(jobId) => future { @@ -329,19 +348,18 @@ class JobManager( log.info(s"Attempting to recover job $jobId.") - val jobGraph = submittedJobGraphs.recoverJobGraph(jobId) + val submittedJobGraphOption = submittedJobGraphs.recoverJobGraph(jobId) - if (jobGraph.isDefined) { - if (!leaderElectionService.hasLeadership()) { - // we've lost leadership. mission: abort. - log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.") - } - else { - recoverJobGraph(jobGraph.get) - } - } - else { - log.warn(s"Failed to recover job graph ${jobId}.") + submittedJobGraphOption match { + case Some(submittedJobGraph) => + if (!leaderElectionService.hasLeadership()) { + // we've lost leadership. mission: abort. + log.warn(s"Lost leadership during recovery. Aborting recovery of $jobId.") + } + else { + self ! decorateMessage(RecoverSubmittedJob(submittedJobGraph)) + } + case None => log.warn(s"Failed to recover job graph $jobId.") } } }(context.dispatcher) @@ -363,7 +381,10 @@ class JobManager( else { log.debug(s"Attempting to recover ${jobGraphs.size} job graphs.") - jobGraphs.foreach(recoverJobGraph(_)) + jobGraphs.foreach{ + submittedJobGraph => + self ! decorateMessage(RecoverSubmittedJob(submittedJobGraph)) + } } } }(context.dispatcher) @@ -474,7 +495,7 @@ class JobManager( if (newJobStatus.isTerminalState()) { jobInfo.end = timeStamp - future { + future{ // TODO If removing the JobGraph from the SubmittedJobGraphsStore fails, the job will // linger around and potentially be recovered at a later time. There is nothing we // can do about that, but it should be communicated with the Client. @@ -484,11 +505,11 @@ class JobManager( context.system.scheduler.scheduleOnce(jobInfo.sessionTimeout seconds) { // remove only if no activity occurred in the meantime if (lastActivity == jobInfo.lastActive) { - removeJob(jobID) + self ! decorateMessage(RemoveJob(jobID, true)) } - } + }(context.dispatcher) } else { - removeJob(jobID) + self ! decorateMessage(RemoveJob(jobID, true)) } // is the client waiting for the job result? @@ -540,9 +561,7 @@ class JobManager( }(context.dispatcher) } case None => - future { - removeJob(jobID) - }(context.dispatcher) + self ! decorateMessage(RemoveJob(jobID, true)) } case ScheduleOrUpdateConsumers(jobId, partitionId) => @@ -647,9 +666,7 @@ class JobManager( case Heartbeat(instanceID, metricsReport, accumulators) => log.debug(s"Received hearbeat message from $instanceID.") - Future { - updateAccumulators(accumulators) - }(context.dispatcher) + updateAccumulators(accumulators) instanceManager.reportHeartBeat(instanceID, metricsReport) @@ -672,11 +689,26 @@ class JobManager( case RequestJobManagerStatus => sender() ! decorateMessage(JobManagerStatusAlive) + case RemoveJob(jobID, clearPersistedJob) => + currentJobs.get(jobID) match { + case Some((graph, info)) => + removeJob(graph.getJobID, clearPersistedJob) match { + case Some(futureToComplete) => + futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) :+ futureToComplete) + case None => + } + case None => + } + case RemoveCachedJob(jobID) => currentJobs.get(jobID) match { case Some((graph, info)) => if (graph.getState.isTerminalState) { - removeJob(graph.getJobID) + removeJob(graph.getJobID, true) match { + case Some(futureToComplete) => + futuresToComplete = Some(futuresToComplete.getOrElse(Seq()) :+ futureToComplete) + case None => + } } else { // triggers removal upon completion of job info.sessionAlive = false @@ -762,6 +794,7 @@ class JobManager( jobGraph.getClasspaths, userCodeLoader) + currentJobs.put(jobGraph.getJobID, (graph, jobInfo)) graph } @@ -879,22 +912,6 @@ class JobManager( executionGraph.registerExecutionListener(gateway) executionGraph.registerJobStatusListener(gateway) } - - if (isRecovery) { - executionGraph.restoreLatestCheckpointedState() - } - else { - submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo)) - } - - // Add the job graph only after everything is finished. Otherwise there can be races in - // tests, which check the currentJobs (for example before killing a JM). - if (!currentJobs.contains(jobId)) { - currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo)) - } - - // done with submitting the job - jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID)) } catch { case t: Throwable => @@ -917,20 +934,39 @@ class JobManager( return } - if (leaderElectionService.hasLeadership) { - // There is a small chance that multiple job managers schedule the same job after if they - // try to recover at the same time. This will eventually be noticed, but can not be ruled - // out from the beginning. - - // NOTE: Scheduling the job for execution is a separate action from the job submission. - // The success of submitting the job must be independent from the success of scheduling - // the job. + // execute the recovery/writing the jobGraph into the SubmittedJobGraphStore asynchronously + // because it is a blocking operation + future { try { - log.info(s"Scheduling job $jobId ($jobName).") + if (isRecovery) { + executionGraph.restoreLatestCheckpointedState() + } + else { + submittedJobGraphs.putJobGraph(new SubmittedJobGraph(jobGraph, jobInfo)) + } - executionGraph.scheduleForExecution(scheduler) - } - catch { + jobInfo.client ! decorateMessage(JobSubmitSuccess(jobGraph.getJobID)) + + if (leaderElectionService.hasLeadership) { + // There is a small chance that multiple job managers schedule the same job after if + // they try to recover at the same time. This will eventually be noticed, but can not be + // ruled out from the beginning. + + // NOTE: Scheduling the job for execution is a separate action from the job submission. + // The success of submitting the job must be independent from the success of scheduling + // the job. + log.info(s"Scheduling job $jobId ($jobName).") + + executionGraph.scheduleForExecution(scheduler) + } else { + // Remove the job graph. Otherwise it will be lingering around and possibly removed from + // ZooKeeper by this JM. + self ! decorateMessage(RemoveJob(jobId, false)) + + log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " + + "this. I am not scheduling the job for execution.") + } + } catch { case t: Throwable => try { executionGraph.fail(t) } @@ -940,27 +976,6 @@ class JobManager( } } } - } - else { - // Remove the job graph. Otherwise it will be lingering around and possibly removed from - // ZooKeeper by this JM. - currentJobs.remove(jobId) - - log.warn(s"Submitted job $jobId, but not leader. The other leader needs to recover " + - "this. I am not scheduling the job for execution.") - } - } - } - - /** - * Submits the job if it is not already one of our current jobs. - * - * @param jobGraph Job to recover - */ - private def recoverJobGraph(jobGraph: SubmittedJobGraph): Unit = { - if (!currentJobs.contains(jobGraph.getJobId)) { - future { - submitJob(jobGraph.getJobGraph(), jobGraph.getJobInfo(), isRecovery = true) }(context.dispatcher) } } @@ -1170,20 +1185,24 @@ class JobManager( * might block. Therefore be careful not to block the actor thread. * * @param jobID ID of the job to remove and archive + * @param removeJobFromStateBackend true if the job shall be archived and removed from the state + * backend */ - private def removeJob(jobID: JobID): Unit = { - currentJobs.synchronized { - // Don't remove the job yet... - currentJobs.get(jobID) match { - case Some((eg, _)) => - try { - // ...otherwise, we can have lingering resources when there is a concurrent shutdown - // and the ZooKeeper client is closed. Not removing the job immediately allow the - // shutdown to release all resources. - submittedJobGraphs.removeJobGraph(jobID) - } catch { - case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t) - } + private def removeJob(jobID: JobID, removeJobFromStateBackend: Boolean): Option[Future[Unit]] = { + // Don't remove the job yet... + val futureOption = currentJobs.get(jobID) match { + case Some((eg, _)) => + val result = if (removeJobFromStateBackend) { + val futureOption = Some(future { + try { + // ...otherwise, we can have lingering resources when there is a concurrent shutdown + // and the ZooKeeper client is closed. Not removing the job immediately allow the + // shutdown to release all resources. + submittedJobGraphs.removeJobGraph(jobID) + } catch { + case t: Throwable => log.error(s"Could not remove submitted job graph $jobID.", t) + } + }(context.dispatcher)) try { eg.prepareForArchiving() @@ -1194,9 +1213,15 @@ class JobManager( "archiving.", t) } - currentJobs.remove(jobID) - case None => - } + futureOption + } else { + None + } + + currentJobs.remove(jobID) + + result + case None => None } try { @@ -1205,6 +1230,8 @@ class JobManager( case t: Throwable => log.error(s"Could not properly unregister job $jobID form the library cache.", t) } + + futureOption } /** Fails all currently running jobs and empties the list of currently running jobs. If the @@ -1212,26 +1239,35 @@ class JobManager( * * @param cause Cause for the cancelling. */ - private def cancelAndClearEverything(cause: Throwable) { - for ((jobID, (eg, jobInfo)) <- currentJobs) { - try { - submittedJobGraphs.removeJobGraph(jobID) - } - catch { - case t: Throwable => { - log.error("Error during submitted job graph clean up.", t) + private def cancelAndClearEverything( + cause: Throwable, + removeJobFromStateBackend: Boolean) + : Seq[Future[Unit]] = { + val futures = for ((jobID, (eg, jobInfo)) <- currentJobs) yield { + future { + if (removeJobFromStateBackend) { + try { + submittedJobGraphs.removeJobGraph(jobID) + } + catch { + case t: Throwable => { + log.error("Error during submitted job graph clean up.", t) + } + } } - } - eg.fail(cause) + eg.fail(cause) - if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) { - jobInfo.client ! decorateMessage( - Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause))) - } + if (jobInfo.listeningBehaviour != ListeningBehaviour.DETACHED) { + jobInfo.client ! decorateMessage( + Failure(new JobExecutionException(jobID, "All jobs are cancelled and cleared.", cause))) + } + }(context.dispatcher) } currentJobs.clear() + + futures.toSeq } override def grantLeadership(newLeaderSessionID: UUID): Unit = { @@ -1286,7 +1322,9 @@ class JobManager( case accumulatorEvent => currentJobs.get(accumulatorEvent.getJobID) match { case Some((jobGraph, jobInfo)) => - jobGraph.updateAccumulators(accumulatorEvent) + future { + jobGraph.updateAccumulators(accumulatorEvent) + }(context.dispatcher) case None => // ignore accumulator values for old job } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala index d77662242e198..8097bdce89ff4 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGra import org.apache.flink.runtime.instance.{InstanceID, Instance} import org.apache.flink.runtime.io.network.partition.ResultPartitionID import org.apache.flink.runtime.jobgraph.{IntermediateDataSetID, JobGraph, JobStatus, JobVertexID} +import org.apache.flink.runtime.jobmanager.SubmittedJobGraph import org.apache.flink.runtime.util.SerializedThrowable import scala.collection.JavaConverters._ @@ -72,6 +73,14 @@ object JobManagerMessages { */ case class RecoverJob(jobId: JobID) extends RequiresLeaderSessionID + /** + * Triggers the submission of the recovered job + * + * @param submittedJobGraph Contains the submitted JobGraph and the associated JobInfo + */ + case class RecoverSubmittedJob(submittedJobGraph: SubmittedJobGraph) + extends RequiresLeaderSessionID + /** * Triggers recovery of all available jobs. */ @@ -286,6 +295,14 @@ object JobManagerMessages { */ case class JobNotFound(jobID: JobID) extends JobResponse with JobStatusResponse + /** Triggers the removal of the job with the given job ID + * + * @param jobID + * @param removeJobFromStateBackend true if the job has properly finished + */ + case class RemoveJob(jobID: JobID, removeJobFromStateBackend: Boolean = true) + extends RequiresLeaderSessionID + /** * Removes the job belonging to the job identifier from the job manager and archives it. * @param jobID The job identifier diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java index ac250bd82a1e9..e6156e5312bd6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerSubmittedJobGraphsRecoveryITCase.java @@ -194,7 +194,9 @@ public void testSubmitJobToNonLeader() throws Exception { JobManagerActorTestUtils.waitForJobStatus(jobGraph.getJobID(), JobStatus.RUNNING, leadingJobManager, deadline.timeLeft()); - // Make sure that the **non-leading** JM has actually removed the job graph from her + log.info("Wait that the non-leader removes the submitted job."); + + // Make sure that the **non-leading** JM has actually removed the job graph from its // local state. boolean success = false; while (!success && deadline.hasTimeLeft()) { @@ -205,6 +207,7 @@ public void testSubmitJobToNonLeader() throws Exception { success = true; } else { + log.info(((JobManagerMessages.CurrentJobStatus)jobStatusResponse).status().toString()); Thread.sleep(100); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java index 7ae89d16f66b0..94e198805179e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java @@ -26,6 +26,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.util.ZooKeeperUtils; +import java.util.List; + /** * Simple ZooKeeper and CuratorFramework setup for tests. */ @@ -111,6 +113,14 @@ public CuratorFramework getClient() { return client; } + public String getClientNamespace() { + return client.getNamespace(); + } + + public List getChildren(String path) throws Exception { + return client.getChildren().forPath(path); + } + /** * Creates a new client for the started ZooKeeper server/cluster. */ diff --git a/flink-runtime/src/test/resources/log4j-test.properties b/flink-runtime/src/test/resources/log4j-test.properties index 76b237e7e92a2..1ca02aaa9b9ef 100644 --- a/flink-runtime/src/test/resources/log4j-test.properties +++ b/flink-runtime/src/test/resources/log4j-test.properties @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=OFF, console +log4j.rootLogger=INFO, console # ----------------------------------------------------------------------------- # Console (use 'console') diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala index 3a252f8cfe9ec..0f800c992969b 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerITCase.scala @@ -657,7 +657,7 @@ class JobManagerITCase(_system: ActorSystem) jm.tell(SubmitJob(jobGraph2, ListeningBehaviour.EXECUTION_RESULT), self) expectMsg(JobSubmitSuccess(jobGraph2.getJobID)) - // job stil running + // job still running jm.tell(RemoveCachedJob(jobGraph2.getJobID), self) expectMsgType[JobResultSuccess] diff --git a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml new file mode 100644 index 0000000000000..c0a2adc5c8c9d --- /dev/null +++ b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml @@ -0,0 +1,78 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-shaded-curator + 0.10-SNAPSHOT + .. + + + flink-shaded-curator-recipes + flink-shaded-curator-recipes + + jar + + + + + org.apache.curator + curator-recipes + ${curator.version} + + + + + com.google.guava + guava + ${guava.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + log4j + org.slf4j:slf4j-log4j12 + + + + + + + + + diff --git a/flink-shaded-curator/flink-shaded-curator-test/pom.xml b/flink-shaded-curator/flink-shaded-curator-test/pom.xml new file mode 100644 index 0000000000000..2700c0c3fa6ba --- /dev/null +++ b/flink-shaded-curator/flink-shaded-curator-test/pom.xml @@ -0,0 +1,86 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-shaded-curator + 0.10-SNAPSHOT + .. + + + flink-shaded-curator-test + flink-shaded-curator-test + + jar + + + + + org.apache.curator + curator-test + ${curator.version} + + + + + com.google.guava + guava + ${guava.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + log4j + org.slf4j:slf4j-log4j12 + + + org.apache.curator:curator-test + + + + + org.apache.curator + org.apache.flink.shaded.org.apache.curator + + + + + + + + + diff --git a/flink-shaded-curator/pom.xml b/flink-shaded-curator/pom.xml index ac62cc8c97b14..29d646184007f 100644 --- a/flink-shaded-curator/pom.xml +++ b/flink-shaded-curator/pom.xml @@ -1,22 +1,21 @@ + ~ 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. + --> @@ -29,50 +28,13 @@ under the License. .. + + flink-shaded-curator-recipes + flink-shaded-curator-test + + flink-shaded-curator flink-shaded-curator - jar - - - - - org.apache.curator - curator-recipes - ${curator.version} - - - - - com.google.guava - guava - ${guava.version} - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - log4j - org.slf4j:slf4j-log4j12 - - - - - - - - + pom diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index b4bddef7485d5..587b6d1003a07 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -140,6 +140,13 @@ under the License. test + + org.apache.flink + flink-shaded-curator-test + ${project.version} + test + + org.scalatest scalatest_${scala.binary.version} diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java index a0c831238c474..2cdf83c20e119 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java @@ -523,22 +523,19 @@ private void submitJobGraph( } private void checkCleanRecoveryState(Configuration config) throws Exception { - LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + LOG.info("Checking " + ZooKeeper.getClientNamespace() + ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); - List jobGraphs = ZooKeeper.getClient().getChildren() - .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); + List jobGraphs = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_JOBGRAPHS_PATH); assertEquals("Unclean job graphs: " + jobGraphs, 0, jobGraphs.size()); - LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + LOG.info("Checking " + ZooKeeper.getClientNamespace() + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); - List checkpoints = ZooKeeper.getClient().getChildren() - .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); + List checkpoints = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINTS_PATH); assertEquals("Unclean checkpoints: " + checkpoints, 0, checkpoints.size()); - LOG.info("Checking " + ZooKeeper.getClient().getNamespace() + + LOG.info("Checking " + ZooKeeper.getClientNamespace() + ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); - List checkpointCounter = ZooKeeper.getClient().getChildren() - .forPath(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); + List checkpointCounter = ZooKeeper.getChildren(ConfigConstants.DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH); assertEquals("Unclean checkpoint counter: " + checkpointCounter, 0, checkpointCounter.size()); LOG.info("ZooKeeper state is clean"); diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 94d0a81a75b38..a05621a985d5e 100644 --- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -22,6 +22,7 @@ import akka.actor.PoisonPill; import akka.testkit.JavaTestKit; import org.apache.curator.test.TestingServer; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; @@ -37,7 +38,9 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import scala.concurrent.duration.FiniteDuration; import java.io.File; @@ -53,6 +56,9 @@ public class YARNHighAvailabilityITCase extends YarnTestBase { private static final int numberApplicationAttempts = 10; + @Rule + public TemporaryFolder tmp = new TemporaryFolder(); + @BeforeClass public static void setup() { actorSystem = AkkaUtils.createDefaultActorSystem(); @@ -102,9 +108,14 @@ public void testMultipleAMKill() throws Exception { String confDirPath = System.getenv("FLINK_CONF_DIR"); flinkYarnClient.setConfigurationDirectory(confDirPath); + String fsStateHandlePath = tmp.getRoot().getPath(); + flinkYarnClient.setFlinkConfigurationObject(GlobalConfiguration.getConfiguration()); flinkYarnClient.setDynamicPropertiesEncoded("recovery.mode=zookeeper@@ha.zookeeper.quorum=" + - zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts); + zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts + + "@@" + ConfigConstants.STATE_BACKEND + "=FILESYSTEM" + + "@@" + ConfigConstants.STATE_BACKEND_FS_DIR + "=" + fsStateHandlePath + "/checkpoints" + + "@@" + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH + "=" + fsStateHandlePath + "/recovery"); flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml")); AbstractFlinkYarnCluster yarnCluster = null; diff --git a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala index 83d1f3c2fa54d..fa7003953ac19 100644 --- a/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala +++ b/flink-yarn-tests/src/main/scala/org/apache/flink/yarn/TestingYarnJobManager.scala @@ -21,8 +21,10 @@ package org.apache.flink.yarn import akka.actor.ActorRef import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.StreamingMode +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.instance.InstanceManager +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore import org.apache.flink.runtime.jobmanager.scheduler.Scheduler import org.apache.flink.runtime.leaderelection.LeaderElectionService import org.apache.flink.runtime.testingUtils.TestingJobManagerLike @@ -60,7 +62,9 @@ class TestingYarnJobManager( delayBetweenRetries: Long, timeout: FiniteDuration, mode: StreamingMode, - leaderElectionService: LeaderElectionService) + leaderElectionService: LeaderElectionService, + submittedJobGraphs : SubmittedJobGraphStore, + checkpointRecoveryFactory : CheckpointRecoveryFactory) extends YarnJobManager( flinkConfiguration, executionContext, @@ -72,7 +76,9 @@ class TestingYarnJobManager( delayBetweenRetries, timeout, mode, - leaderElectionService) + leaderElectionService, + submittedJobGraphs, + checkpointRecoveryFactory) with TestingJobManagerLike { override val taskManagerRunnerClass = classOf[TestingYarnTaskManagerRunner] From c8e63364f6289ea02f2871c65bf2070683ff0598 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Mon, 5 Oct 2015 14:30:46 +0200 Subject: [PATCH 6/7] [FLINK-2805] [blobmanager] Write JARs to file state backend for recovery Move StateBackend enum to top level and org.apache.flink.runtime.state Abstract blob store in blob server for recovery This closes #1227. --- .../apache/flink/runtime/blob/BlobServer.java | 105 ++++++++-- .../runtime/blob/BlobServerConnection.java | 52 +++-- .../apache/flink/runtime/blob/BlobStore.java | 97 +++++++++ .../apache/flink/runtime/blob/BlobUtils.java | 75 ++++++- .../runtime/blob/FileSystemBlobStore.java | 186 ++++++++++++++++++ .../flink/runtime/blob/VoidBlobStore.java | 61 ++++++ .../runtime/jobmanager/RecoveryMode.java | 12 +- .../runtime/blob/BlobRecoveryITCase.java | 159 +++++++++++++++ .../BlobLibraryCacheRecoveryITCase.java | 176 +++++++++++++++++ 9 files changed, 874 insertions(+), 49 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index ef2ef617e6c6c..d0bed8c04632b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -18,6 +18,14 @@ package org.apache.flink.runtime.blob; +import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -30,13 +38,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.io.FileUtils; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.api.common.JobID; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static com.google.common.base.Preconditions.checkNotNull; /** * This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and @@ -57,31 +59,56 @@ public class BlobServer extends Thread implements BlobService { /** Indicates whether a shutdown of server component has been requested. */ private final AtomicBoolean shutdownRequested = new AtomicBoolean(); - /** Shutdown hook thread to ensure deletion of the storage directory. */ - private final Thread shutdownHook; - /** Is the root directory for file storage */ private final File storageDir; + /** Blob store for recovery */ + private final BlobStore blobStore; + /** Set of currently running threads */ private final Set activeConnections = new HashSet(); /** The maximum number of concurrent connections */ private final int maxConnections; + /** + * Shutdown hook thread to ensure deletion of the storage directory (or null if + * the configured recovery mode does not equal{@link RecoveryMode#STANDALONE}) + */ + private final Thread shutdownHook; + /** * Instantiates a new BLOB server and binds it to a free network port. - * + * * @throws IOException * thrown if the BLOB server cannot bind to a free network port */ public BlobServer(Configuration config) throws IOException { + checkNotNull(config, "Configuration"); + + RecoveryMode recoveryMode = RecoveryMode.fromConfig(config); // configure and create the storage directory String storageDirectory = config.getString(ConfigConstants.BLOB_STORAGE_DIRECTORY_KEY, null); this.storageDir = BlobUtils.initStorageDirectory(storageDirectory); LOG.info("Created BLOB server storage directory {}", storageDir); + // No recovery. + if (recoveryMode == RecoveryMode.STANDALONE) { + this.blobStore = new VoidBlobStore(); + } + // Recovery. Check that everything has been setup correctly. This is not clean, but it's + // better to resolve this with some upcoming changes to the state backend setup. + else if (config.containsKey(ConfigConstants.STATE_BACKEND) && + config.containsKey(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH)) { + + this.blobStore = new FileSystemBlobStore(config); + } + // Fallback. + else { + this.blobStore = new VoidBlobStore(); + } + // configure the maximum number of concurrent connections final int maxConnections = config.getInteger( ConfigConstants.BLOB_FETCH_CONCURRENT_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT); @@ -102,8 +129,13 @@ public BlobServer(Configuration config) throws IOException { backlog = ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG; } - // Add shutdown hook to delete storage directory - this.shutdownHook = BlobUtils.addShutdownHook(this, LOG); + if (recoveryMode == RecoveryMode.STANDALONE) { + // Add shutdown hook to delete storage directory + this.shutdownHook = BlobUtils.addShutdownHook(this, LOG); + } + else { + this.shutdownHook = null; + } // start the server try { @@ -132,37 +164,43 @@ public BlobServer(Configuration config) throws IOException { * Returns a file handle to the file associated with the given blob key on the blob * server. * + *

This is only called from the {@link BlobServerConnection} + * * @param key identifying the file * @return file handle to the file */ - public File getStorageLocation(BlobKey key) { + File getStorageLocation(BlobKey key) { return BlobUtils.getStorageLocation(storageDir, key); } /** * Returns a file handle to the file identified by the given jobID and key. * + *

This is only called from the {@link BlobServerConnection} + * * @param jobID to which the file is associated * @param key to identify the file within the job context * @return file handle to the file */ - public File getStorageLocation(JobID jobID, String key) { + File getStorageLocation(JobID jobID, String key) { return BlobUtils.getStorageLocation(storageDir, jobID, key); } /** * Method which deletes all files associated with the given jobID. * + *

This is only called from the {@link BlobServerConnection} + * * @param jobID all files associated to this jobID will be deleted * @throws IOException */ - public void deleteJobDirectory(JobID jobID) throws IOException { + void deleteJobDirectory(JobID jobID) throws IOException { BlobUtils.deleteJobDirectory(storageDir, jobID); } /** * Returns a temporary file inside the BLOB server's incoming directory. - * + * * @return a temporary file inside the BLOB server's incoming directory */ File createTemporaryFilename() { @@ -170,6 +208,13 @@ File createTemporaryFilename() { String.format("temp-%08d", tempFileCounter.getAndIncrement())); } + /** + * Returns the blob store. + */ + BlobStore getBlobStore() { + return blobStore; + } + @Override public void run() { try { @@ -245,6 +290,9 @@ public void shutdown() { LOG.error("BLOB server failed to properly clean up its storage directory."); } + // Clean up the recovery directory + blobStore.cleanUp(); + // Remove shutdown hook to prevent resource leaks, unless this is invoked by the // shutdown hook itself if (shutdownHook != null && shutdownHook != Thread.currentThread()) { @@ -282,11 +330,26 @@ public URL getURL(BlobKey requiredBlob) throws IOException { final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob); - if (!localFile.exists()) { - throw new FileNotFoundException("File " + localFile.getCanonicalPath() + " does not exist."); - } else { + if (localFile.exists()) { return localFile.toURI().toURL(); } + else { + try { + // Try the blob store + blobStore.get(requiredBlob, localFile); + } + catch (Exception e) { + throw new IOException("Failed to copy from blob store.", e); + } + + if (localFile.exists()) { + return localFile.toURI().toURL(); + } + else { + throw new FileNotFoundException("Local file " + localFile + " does not exist " + + "and failed to copy from blob store."); + } + } } /** @@ -305,6 +368,8 @@ public void delete(BlobKey key) throws IOException { LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath()); } } + + blobStore.delete(key); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java index 793a9d6b1d273..d7bba8f754f7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java @@ -18,6 +18,12 @@ package org.apache.flink.runtime.blob; +import com.google.common.io.Files; +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.InstantiationUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.EOFException; import java.io.File; import java.io.FileInputStream; @@ -29,28 +35,21 @@ import java.net.SocketException; import java.security.MessageDigest; -import com.google.common.io.Files; -import org.apache.flink.api.common.JobID; -import org.apache.flink.util.InstantiationUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE; +import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION; +import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.JOB_ID_SCOPE; +import static org.apache.flink.runtime.blob.BlobServerProtocol.MAX_KEY_LENGTH; import static org.apache.flink.runtime.blob.BlobServerProtocol.NAME_ADDRESSABLE; +import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; +import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR; +import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY; import static org.apache.flink.runtime.blob.BlobUtils.closeSilently; import static org.apache.flink.runtime.blob.BlobUtils.readFully; import static org.apache.flink.runtime.blob.BlobUtils.readLength; import static org.apache.flink.runtime.blob.BlobUtils.writeLength; -import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; -import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION; -import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; -import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; -import static org.apache.flink.runtime.blob.BlobServerProtocol.MAX_KEY_LENGTH; -import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_OKAY; -import static org.apache.flink.runtime.blob.BlobServerProtocol.RETURN_ERROR; - /** * A BLOB connection handles a series of requests from a particular BLOB client. */ @@ -181,10 +180,18 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) JobID jobID = JobID.fromByteArray(jidBytes); String key = readKey(buf, inputStream); blobFile = this.blobServer.getStorageLocation(jobID, key); + + if (!blobFile.exists()) { + blobServer.getBlobStore().get(jobID, key, blobFile); + } } else if (contentAddressable == CONTENT_ADDRESSABLE) { final BlobKey key = BlobKey.readFromInputStream(inputStream); blobFile = blobServer.getStorageLocation(key); + + if (!blobFile.exists()) { + blobServer.getBlobStore().get(key, blobFile); + } } else { throw new IOException("Unknown type of BLOB addressing."); @@ -194,6 +201,7 @@ else if (contentAddressable == CONTENT_ADDRESSABLE) { if (!blobFile.exists()) { throw new IOException("Cannot find required BLOB at " + blobFile.getAbsolutePath()); } + if (blobFile.length() > Integer.MAX_VALUE) { throw new IOException("BLOB size exceeds the maximum size (2 GB)."); } @@ -220,8 +228,7 @@ else if (contentAddressable == CONTENT_ADDRESSABLE) { int blobLen = (int) blobFile.length(); writeLength(blobLen, outputStream); - FileInputStream fis = new FileInputStream(blobFile); - try { + try (FileInputStream fis = new FileInputStream(blobFile)) { int bytesRemaining = blobLen; while (bytesRemaining > 0) { int read = fis.read(buf); @@ -231,8 +238,6 @@ else if (contentAddressable == CONTENT_ADDRESSABLE) { outputStream.write(buf, 0, read); bytesRemaining -= read; } - } finally { - fis.close(); } } catch (SocketException e) { @@ -314,6 +319,9 @@ else if (contentAddressable == CONTENT_ADDRESSABLE) { File storageFile = this.blobServer.getStorageLocation(jobID, key); Files.move(incomingFile, storageFile); incomingFile = null; + + blobServer.getBlobStore().put(storageFile, jobID, key); + outputStream.write(RETURN_OKAY); } else { @@ -322,6 +330,8 @@ else if (contentAddressable == CONTENT_ADDRESSABLE) { Files.move(incomingFile, storageFile); incomingFile = null; + blobServer.getBlobStore().put(storageFile, blobKey); + // Return computed key to client for validation outputStream.write(RETURN_OKAY); blobKey.writeToOutputStream(outputStream); @@ -379,6 +389,8 @@ private void delete(InputStream inputStream, OutputStream outputStream, byte[] b if (blobFile.exists() && !blobFile.delete()) { throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath()); } + + blobServer.getBlobStore().delete(key); } else if (type == NAME_ADDRESSABLE) { byte[] jidBytes = new byte[JobID.SIZE]; @@ -391,6 +403,8 @@ else if (type == NAME_ADDRESSABLE) { if (blobFile.exists() && !blobFile.delete()) { throw new IOException("Cannot delete BLOB file " + blobFile.getAbsolutePath()); } + + blobServer.getBlobStore().delete(jobID, key); } else if (type == JOB_ID_SCOPE) { byte[] jidBytes = new byte[JobID.SIZE]; @@ -398,6 +412,8 @@ else if (type == JOB_ID_SCOPE) { JobID jobID = JobID.fromByteArray(jidBytes); blobServer.deleteJobDirectory(jobID); + + blobServer.getBlobStore().deleteAll(jobID); } else { throw new IOException("Unrecognized addressing type: " + type); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java new file mode 100644 index 0000000000000..1e72d9112d4e6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java @@ -0,0 +1,97 @@ +/* + * 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.blob; + +import org.apache.flink.api.common.JobID; + +import java.io.File; + +/** + * A blob store. + */ +interface BlobStore { + + /** + * Copies the local file to the blob store. + * + * @param localFile The file to copy + * @param blobKey The ID for the file in the blob store + * @throws Exception If the copy fails + */ + void put(File localFile, BlobKey blobKey) throws Exception; + + /** + * Copies a local file to the blob store. + * + *

The job ID and key make up a composite key for the file. + * + * @param localFile The file to copy + * @param jobId The JobID part of ID for the file in the blob store + * @param key The String part of ID for the file in the blob store + * @throws Exception If the copy fails + */ + void put(File localFile, JobID jobId, String key) throws Exception; + + /** + * Copies a blob to a local file. + * + * @param blobKey The blob ID + * @param localFile The local file to copy to + * @throws Exception If the copy fails + */ + void get(BlobKey blobKey, File localFile) throws Exception; + + /** + * Copies a blob to a local file. + * + * @param jobId The JobID part of ID for the blob + * @param key The String part of ID for the blob + * @param localFile The local file to copy to + * @throws Exception If the copy fails + */ + void get(JobID jobId, String key, File localFile) throws Exception; + + /** + * Deletes a blob. + * + * @param blobKey The blob ID + */ + void delete(BlobKey blobKey); + + /** + * Deletes a blob. + * + * @param jobId The JobID part of ID for the blob + * @param key The String part of ID for the blob + */ + void delete(JobID jobId, String key); + + /** + * Deletes blobs. + * + * @param jobId The JobID part of all blobs to delete + */ + void deleteAll(JobID jobId); + + /** + * Cleans up the store and deletes all blobs. + */ + void cleanUp(); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index c47ecf25c4475..d8f744bc24bf9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -21,14 +21,19 @@ import com.google.common.io.BaseEncoding; import org.apache.commons.io.FileUtils; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.util.IOUtils; import org.slf4j.Logger; import java.io.EOFException; import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.net.Socket; +import java.net.URI; import java.nio.charset.Charset; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; @@ -49,12 +54,12 @@ public class BlobUtils { /** * The prefix of all BLOB files stored by the BLOB server. */ - private static final String BLOB_FILE_PREFIX = "blob_"; + static final String BLOB_FILE_PREFIX = "blob_"; /** * The prefix of all job-specific directories created by the BLOB server. */ - private static final String JOB_DIR_PREFIX = "job_"; + static final String JOB_DIR_PREFIX = "job_"; /** * The default character set to translate between characters and bytes. @@ -103,7 +108,7 @@ static File initStorageDirectory(String storageDirectory) { static File getIncomingDirectory(File storageDir) { final File incomingDir = new File(storageDir, "incoming"); - if (!incomingDir.exists() && !incomingDir.mkdir()) { + if (!incomingDir.exists() && !incomingDir.mkdirs()) { throw new RuntimeException("Cannot create directory for incoming files " + incomingDir.getAbsolutePath()); } @@ -119,7 +124,7 @@ static File getIncomingDirectory(File storageDir) { private static File getCacheDirectory(File storageDir) { final File cacheDirectory = new File(storageDir, "cache"); - if (!cacheDirectory.exists() && !cacheDirectory.mkdir()) { + if (!cacheDirectory.exists() && !cacheDirectory.mkdirs()) { throw new RuntimeException("Could not create cache directory '" + cacheDirectory.getAbsolutePath() + "'."); } @@ -174,7 +179,7 @@ private static File getJobDirectory(File storageDir, JobID jobID) { * the user's key for a BLOB * @return the internal name for the BLOB as used by the BLOB server */ - private static String encodeKey(String key) { + static String encodeKey(String key) { return BaseEncoding.base64().encode(key.getBytes(DEFAULT_CHARSET)); } @@ -326,6 +331,66 @@ static void closeSilently(Socket socket, Logger LOG) { } } + /** + * Returns the path for the given blob key. + * + *

The returned path can be used with the state backend for recovery purposes. + * + *

This follows the same scheme as {@link #getStorageLocation(File, BlobKey)}. + */ + static String getRecoveryPath(String basePath, BlobKey blobKey) { + // format: $base/cache/blob_$key + return String.format("%s/cache/%s", basePath, BLOB_FILE_PREFIX + blobKey.toString()); + } + + /** + * Returns the path for the given job ID and key. + * + *

The returned path can be used with the state backend for recovery purposes. + * + *

This follows the same scheme as {@link #getStorageLocation(File, JobID, String)}. + */ + static String getRecoveryPath(String basePath, JobID jobId, String key) { + // format: $base/job_$id/blob_$key + return String.format("%s/%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString(), + BLOB_FILE_PREFIX + encodeKey(key)); + } + + /** + * Returns the path for the given job ID. + * + *

The returned path can be used with the state backend for recovery purposes. + */ + static String getRecoveryPath(String basePath, JobID jobId) { + return String.format("%s/%s", basePath, JOB_DIR_PREFIX + jobId.toString()); + } + + /** + * Copies the file from the recovery path to the local file. + */ + static void copyFromRecoveryPath(String recoveryPath, File localBlobFile) throws Exception { + if (recoveryPath == null) { + throw new IllegalStateException("Failed to determine recovery path."); + } + + if (!localBlobFile.createNewFile()) { + throw new IllegalStateException("Failed to create new local file to copy to"); + } + + URI uri = new URI(recoveryPath); + Path path = new Path(recoveryPath); + + if (FileSystem.get(uri).exists(path)) { + try (InputStream is = FileSystem.get(uri).open(path)) { + FileOutputStream fos = new FileOutputStream(localBlobFile); + IOUtils.copyBytes(is, fos); // closes the streams + } + } + else { + throw new IOException("Cannot find required BLOB at '" + recoveryPath + "' for recovery."); + } + } + /** * Private constructor to prevent instantiation. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java new file mode 100644 index 0000000000000..8a037ad821a69 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java @@ -0,0 +1,186 @@ +/* + * 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.blob; + +import com.google.common.io.Files; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.util.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.URI; +import java.net.URISyntaxException; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Blob store backed by {@link FileSystem}. + */ +class FileSystemBlobStore implements BlobStore { + + private static final Logger LOG = LoggerFactory.getLogger(FileSystemBlobStore.class); + + /** The base path of the blob store */ + private final String basePath; + + FileSystemBlobStore(Configuration config) throws IOException { + StateBackend stateBackend = StateBackend.fromConfig(config); + + if (stateBackend == StateBackend.FILESYSTEM) { + String stateBackendBasePath = config.getString( + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, ""); + + if (stateBackendBasePath.equals("")) { + throw new IllegalConfigurationException(String.format("Missing configuration for " + + "file system state backend recovery path. Please specify via " + + "'%s' key.", ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH)); + } + + stateBackendBasePath += "/blob"; + + this.basePath = stateBackendBasePath; + + try { + FileSystem.get(new URI(basePath)).mkdirs(new Path(basePath)); + } + catch (URISyntaxException e) { + throw new IOException(e); + } + + LOG.info("Created blob directory {}.", basePath); + } + else { + // Nothing else support at the moment + throw new IllegalConfigurationException( + String.format("Illegal state backend " + + "configuration '%s'. Please configure '%s' as state " + + "backend and specify the recovery path via '%s' key.", + stateBackend, StateBackend.FILESYSTEM, + ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH)); + } + } + + // - Put ------------------------------------------------------------------ + + @Override + public void put(File localFile, BlobKey blobKey) throws Exception { + put(localFile, BlobUtils.getRecoveryPath(basePath, blobKey)); + } + + @Override + public void put(File localFile, JobID jobId, String key) throws Exception { + put(localFile, BlobUtils.getRecoveryPath(basePath, jobId, key)); + } + + private void put(File fromFile, String toBlobPath) throws Exception { + try (OutputStream os = FileSystem.get(new URI(toBlobPath)) + .create(new Path(toBlobPath), true)) { + + LOG.debug("Copying from {} to {}.", fromFile, toBlobPath); + Files.copy(fromFile, os); + } + } + + // - Get ------------------------------------------------------------------ + + @Override + public void get(BlobKey blobKey, File localFile) throws Exception { + get(BlobUtils.getRecoveryPath(basePath, blobKey), localFile); + } + + @Override + public void get(JobID jobId, String key, File localFile) throws Exception { + get(BlobUtils.getRecoveryPath(basePath, jobId, key), localFile); + } + + private void get(String fromBlobPath, File toFile) throws Exception { + checkNotNull(fromBlobPath, "Blob path"); + checkNotNull(toFile, "File"); + + if (!toFile.exists() && !toFile.createNewFile()) { + throw new IllegalStateException("Failed to create target file to copy to"); + } + + final URI fromUri = new URI(fromBlobPath); + final Path fromPath = new Path(fromBlobPath); + + if (FileSystem.get(fromUri).exists(fromPath)) { + try (InputStream is = FileSystem.get(fromUri).open(fromPath)) { + FileOutputStream fos = new FileOutputStream(toFile); + + LOG.debug("Copying from {} to {}.", fromBlobPath, toFile); + IOUtils.copyBytes(is, fos); // closes the streams + } + } + else { + throw new IOException(fromBlobPath + " does not exist."); + } + } + + // - Delete --------------------------------------------------------------- + + @Override + public void delete(BlobKey blobKey) { + delete(BlobUtils.getRecoveryPath(basePath, blobKey)); + } + + @Override + public void delete(JobID jobId, String key) { + delete(BlobUtils.getRecoveryPath(basePath, jobId, key)); + } + + @Override + public void deleteAll(JobID jobId) { + delete(BlobUtils.getRecoveryPath(basePath, jobId)); + } + + private void delete(String blobPath) { + try { + LOG.debug("Deleting {}.", blobPath); + + FileSystem.get(new URI(blobPath)).delete(new Path(blobPath), true); + } + catch (Exception e) { + LOG.warn("Failed to delete blob at " + blobPath); + } + } + + @Override + public void cleanUp() { + try { + LOG.debug("Cleaning up {}.", basePath); + + FileSystem.get(new URI(basePath)).delete(new Path(basePath), true); + } + catch (Exception e) { + LOG.error("Failed to clean up recovery directory."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java new file mode 100644 index 0000000000000..1b71add29b631 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java @@ -0,0 +1,61 @@ +/* + * 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.blob; + +import org.apache.flink.api.common.JobID; + +import java.io.File; + +/** + * A blob store doing nothing. + */ +class VoidBlobStore implements BlobStore { + + @Override + public void put(File localFile, BlobKey blobKey) throws Exception { + } + + @Override + public void put(File localFile, JobID jobId, String key) throws Exception { + } + + @Override + public void get(BlobKey blobKey, File localFile) throws Exception { + } + + @Override + public void get(JobID jobId, String key, File localFile) throws Exception { + } + + @Override + public void delete(BlobKey blobKey) { + } + + @Override + public void delete(JobID jobId, String key) { + } + + @Override + public void deleteAll(JobID jobId) { + } + + @Override + public void cleanUp() { + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java index 17322d8f248af..077e34da3775a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/RecoveryMode.java @@ -24,11 +24,11 @@ /** * Recovery mode for Flink's cluster execution. Currently supported modes are: * - * - Standalone: No recovery from JobManager failures - * - ZooKeeper: JobManager high availability via ZooKeeper - * ZooKeeper is used to select a leader among a group of JobManager. This JobManager - * is responsible for the job execution. Upon failure of the leader a new leader is elected - * which will take over the responsibilities of the old leader + * - Standalone: No recovery from JobManager failures + * - ZooKeeper: JobManager high availability via ZooKeeper + * ZooKeeper is used to select a leader among a group of JobManager. This JobManager + * is responsible for the job execution. Upon failure of the leader a new leader is elected + * which will take over the responsibilities of the old leader */ public enum RecoveryMode { STANDALONE, @@ -69,4 +69,4 @@ public static boolean isHighAvailabilityModeActivated(Configuration configuratio return false; } } -} +} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java new file mode 100644 index 0000000000000..0e324a84874fa --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java @@ -0,0 +1,159 @@ +/* + * 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.blob; + +import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.InputStream; +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +public class BlobRecoveryITCase { + + private File recoveryDir; + + @Before + public void setUp() throws Exception { + recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir"); + if (!recoveryDir.exists() && !recoveryDir.mkdirs()) { + throw new IllegalStateException("Failed to create temp directory for test"); + } + } + + @After + public void cleanUp() throws Exception { + if (recoveryDir != null) { + FileUtils.deleteDirectory(recoveryDir); + } + } + + /** + * Tests that with {@link RecoveryMode#ZOOKEEPER} distributed JARs are recoverable from any + * participating BlobServer. + */ + @Test + public void testBlobServerRecovery() throws Exception { + Random rand = new Random(); + + BlobServer[] server = new BlobServer[2]; + InetSocketAddress[] serverAddress = new InetSocketAddress[2]; + BlobClient client = null; + + try { + Configuration config = new Configuration(); + config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER"); + config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM"); + config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, recoveryDir.getPath()); + + for (int i = 0; i < server.length; i++) { + server[i] = new BlobServer(config); + serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort()); + } + + client = new BlobClient(serverAddress[0]); + + // Random data + byte[] expected = new byte[1024]; + rand.nextBytes(expected); + + BlobKey[] keys = new BlobKey[2]; + + // Put data + keys[0] = client.put(expected); // Request 1 + keys[1] = client.put(expected, 32, 256); // Request 2 + + JobID[] jobId = new JobID[] { new JobID(), new JobID() }; + String[] testKey = new String[] { "test-key-1", "test-key-2" }; + + client.put(jobId[0], testKey[0], expected); // Request 3 + client.put(jobId[1], testKey[1], expected, 32, 256); // Request 4 + + // Close the client and connect to the other server + client.close(); + client = new BlobClient(serverAddress[1]); + + // Verify request 1 + try (InputStream is = client.get(keys[0])) { + byte[] actual = new byte[expected.length]; + + BlobUtils.readFully(is, actual, 0, expected.length, null); + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], actual[i]); + } + } + + // Verify request 2 + try (InputStream is = client.get(keys[1])) { + byte[] actual = new byte[256]; + BlobUtils.readFully(is, actual, 0, 256, null); + + for (int i = 32, j = 0; i < 256; i++, j++) { + assertEquals(expected[i], actual[j]); + } + } + + // Verify request 3 + try (InputStream is = client.get(jobId[0], testKey[0])) { + byte[] actual = new byte[expected.length]; + BlobUtils.readFully(is, actual, 0, expected.length, null); + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], actual[i]); + } + } + + // Verify request 4 + try (InputStream is = client.get(jobId[1], testKey[1])) { + byte[] actual = new byte[256]; + BlobUtils.readFully(is, actual, 0, 256, null); + + for (int i = 32, j = 0; i < 256; i++, j++) { + assertEquals(expected[i], actual[j]); + } + } + } + finally { + for (BlobServer s : server) { + if (s != null) { + s.shutdown(); + } + } + + if (client != null) { + client.close(); + } + } + + // Verify everything is clean + File[] recoveryFiles = recoveryDir.listFiles(); + assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java new file mode 100644 index 0000000000000..4df8afb290089 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -0,0 +1,176 @@ +/* + * 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.execution.librarycache; + +import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobCache; +import org.apache.flink.runtime.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobmanager.RecoveryMode; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; +import java.net.InetSocketAddress; +import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +public class BlobLibraryCacheRecoveryITCase { + + private File recoveryDir; + + @Before + public void setUp() throws Exception { + recoveryDir = new File(FileUtils.getTempDirectory(), "BlobRecoveryITCaseDir"); + if (!recoveryDir.exists() && !recoveryDir.mkdirs()) { + throw new IllegalStateException("Failed to create temp directory for test"); + } + } + + @After + public void cleanUp() throws Exception { + if (recoveryDir != null) { + FileUtils.deleteDirectory(recoveryDir); + } + } + + /** + * Tests that with {@link RecoveryMode#ZOOKEEPER} distributed JARs are recoverable from any + * participating BlobLibraryCacheManager. + */ + @Test + public void testRecoveryRegisterAndDownload() throws Exception { + Random rand = new Random(); + + BlobServer[] server = new BlobServer[2]; + InetSocketAddress[] serverAddress = new InetSocketAddress[2]; + BlobLibraryCacheManager[] libServer = new BlobLibraryCacheManager[2]; + BlobCache cache = null; + BlobLibraryCacheManager libCache = null; + + try { + Configuration config = new Configuration(); + config.setString(ConfigConstants.RECOVERY_MODE, "ZOOKEEPER"); + config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM"); + config.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, recoveryDir.getPath()); + + for (int i = 0; i < server.length; i++) { + server[i] = new BlobServer(config); + serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort()); + libServer[i] = new BlobLibraryCacheManager(server[i], 3600 * 1000); + } + + // Random data + byte[] expected = new byte[1024]; + rand.nextBytes(expected); + + List keys = new ArrayList<>(2); + + // Upload some data (libraries) + try (BlobClient client = new BlobClient(serverAddress[0])) { + keys.add(client.put(expected)); // Request 1 + keys.add(client.put(expected, 32, 256)); // Request 2 + } + + // The cache + cache = new BlobCache(serverAddress[0], config); + libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); + + // Register uploaded libraries + JobID jobId = new JobID(); + ExecutionAttemptID executionId = new ExecutionAttemptID(); + libServer[0].registerTask(jobId, executionId, keys, Collections.emptyList()); + + // Verify key 1 + File f = libCache.getFile(keys.get(0)); + assertEquals(expected.length, f.length()); + + try (FileInputStream fis = new FileInputStream(f)) { + for (int i = 0; i < expected.length && fis.available() > 0; i++) { + assertEquals(expected[i], (byte) fis.read()); + } + + assertEquals(0, fis.available()); + } + + // Shutdown cache and start with other server + cache.shutdown(); + libCache.shutdown(); + + cache = new BlobCache(serverAddress[1], config); + libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); + + // Verify key 1 + f = libCache.getFile(keys.get(0)); + assertEquals(expected.length, f.length()); + + try (FileInputStream fis = new FileInputStream(f)) { + for (int i = 0; i < expected.length && fis.available() > 0; i++) { + assertEquals(expected[i], (byte) fis.read()); + } + + assertEquals(0, fis.available()); + } + + // Verify key 2 + f = libCache.getFile(keys.get(1)); + assertEquals(256, f.length()); + + try (FileInputStream fis = new FileInputStream(f)) { + for (int i = 0; i < 256 && fis.available() > 0; i++) { + assertEquals(expected[32 + i], (byte) fis.read()); + } + + assertEquals(0, fis.available()); + } + } + finally { + for (BlobServer s : server) { + if (s != null) { + s.shutdown(); + } + } + + if (cache != null) { + cache.shutdown(); + } + + if (libCache != null) { + libCache.shutdown(); + } + } + + // Verify everything is clean + File[] recoveryFiles = recoveryDir.listFiles(); + assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length); + } +} From 1da0d78344fb8d065f74aed68271614b2e5fe10d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 8 Oct 2015 01:52:07 +0200 Subject: [PATCH 7/7] [FLINK-2804] [runtime] Add blocking job submission support for HA The JobClientActor is now repsonsible for receiving the JobStatus updates from a newly elected leader. It uses the LeaderRetrievalService to be notified about new leaders. The actor can only be used to submit a single job to the JM. Once it received a job from the Client it tries to send it to the current leader. If no leader is available, a connection timeout is triggered. If the job could be sent to the JM, a submission timeout is triggered if the JobClientActor does not receive a JobSubmitSuccess message within the timeout interval. If the connection to the leader is lost after having submitted a job, a connection timeout is triggered if the JobClientActor cannot reconnect to another JM within the timeout interval. The JobClient simply awaits on the completion of the returned future to the SubmitJobAndWait message. Added test cases for JobClientActor exceptions This closes #1249. --- .../apache/flink/client/program/Client.java | 108 +++--- .../RemoteExecutorHostnameResolutionTest.java | 6 +- .../flink/runtime/client/JobClient.java | 50 +-- .../flink/runtime/client/JobClientActor.java | 360 +++++++++++++++--- ...ClientActorConnectionTimeoutException.java | 35 ++ ...ClientActorSubmissionTimeoutException.java | 35 ++ .../flink/runtime/jobmanager/JobManager.scala | 2 + .../runtime/messages/JobClientMessages.scala | 26 +- .../runtime/messages/JobManagerMessages.scala | 6 +- .../minicluster/FlinkMiniCluster.scala | 24 +- .../runtime/taskmanager/TaskManager.scala | 7 +- .../client/JobClientActorRecoveryITCase.java | 163 ++++++++ .../runtime/client/JobClientActorTest.java | 228 +++++++++++ .../PartialConsumePipelinedResultTest.java | 13 +- .../JobManagerLeaderElectionTest.java | 25 +- .../LeaderChangeStateCleanupTest.java | 9 +- ...LeaderElectionRetrievalTestingCluster.java | 25 +- .../TestingLeaderElectionService.java | 5 +- .../TestingLeaderRetrievalService.java | 16 + .../testingUtils/TestingJobManagerLike.scala | 15 + .../TestingJobManagerMessages.scala | 9 +- .../ZooKeeperLeaderElectionITCase.java | 117 +++--- .../src/test/resources/log4j-test.properties | 7 +- 23 files changed, 1008 insertions(+), 283 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java index dfb9c1b0339d4..322c73d3f9422 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/Client.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/Client.java @@ -24,8 +24,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.accumulators.AccumulatorHelper; @@ -47,7 +47,6 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.util.LeaderRetrievalUtils; @@ -81,12 +80,15 @@ public class Client { /** The actor system used to communicate with the JobManager */ private final ActorSystem actorSystem; - /** The actor reference to the JobManager */ - private final ActorGateway jobManagerGateway; + /** Configuration of the client */ + private final Configuration config; - /** The timeout for communication between the client and the JobManager */ + /** Timeout for futures */ private final FiniteDuration timeout; - + + /** Lookup timeout for the job manager retrieval service */ + private final FiniteDuration lookupTimeout; + /** * If != -1, this field specifies the total number of available slots on the cluster * connected to the client. @@ -133,6 +135,7 @@ public Client(Configuration config) throws IOException { */ public Client(Configuration config, int maxSlots) throws IOException { + this.config = Preconditions.checkNotNull(config); this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config); this.maxSlots = maxSlots; @@ -144,50 +147,8 @@ public Client(Configuration config, int maxSlots) throws IOException { throw new IOException("Could start client actor system.", e); } - // from here on, we need to make sure the actor system is shut down on error - boolean success = false; - - try { - - FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(config); - this.timeout = AkkaUtils.getTimeout(config); - - LOG.info("Looking up JobManager"); - LeaderRetrievalService leaderRetrievalService; - - try { - leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config); - } catch (Exception e) { - throw new IOException("Could not create the leader retrieval service.", e); - } - - try { - this.jobManagerGateway = LeaderRetrievalUtils.retrieveLeaderGateway( - leaderRetrievalService, - actorSystem, - lookupTimeout); - } catch (LeaderRetrievalException e) { - throw new IOException("Failed to retrieve JobManager gateway", e); - } - - LOG.info("Leading JobManager actor system address is " + this.jobManagerGateway.path()); - - LOG.info("JobManager runs at " + this.jobManagerGateway.path()); - - LOG.info("Communication between client and JobManager will have a timeout of " + this.timeout); - success = true; - } finally { - if (!success) { - try { - this.actorSystem.shutdown(); - - // wait at most for 30 seconds, to work around an occasional akka problem - actorSystem.awaitTermination(new FiniteDuration(30, TimeUnit.SECONDS)); - } catch (Throwable t) { - LOG.error("Shutting down actor system after error caused another error", t); - } - } - } + timeout = AkkaUtils.getTimeout(config); + lookupTimeout = AkkaUtils.getTimeout(config); } // ------------------------------------------------------------------------ // Startup & Shutdown @@ -395,21 +356,30 @@ public JobSubmissionResult runDetached(OptimizedPlan compiledPlan, List lib } public JobExecutionResult runBlocking(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { - LOG.info("Checking and uploading JAR files"); + LeaderRetrievalService leaderRetrievalService; try { - JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout); - } catch (IOException e) { - throw new ProgramInvocationException("Could not upload the program's JAR files to the JobManager.", e); + leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config); + } catch (Exception e) { + throw new ProgramInvocationException("Could not create the leader retrieval service.", e); } + try { this.lastJobID = jobGraph.getJobID(); - return JobClient.submitJobAndWait(actorSystem, jobManagerGateway, jobGraph, timeout, printStatusDuringExecution, classLoader); + return JobClient.submitJobAndWait(actorSystem, leaderRetrievalService, jobGraph, timeout, printStatusDuringExecution, classLoader); } catch (JobExecutionException e) { throw new ProgramInvocationException("The program execution failed: " + e.getMessage(), e); } } public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + ActorGateway jobManagerGateway; + + try { + jobManagerGateway = getJobManagerGateway(); + } catch (Exception e) { + throw new ProgramInvocationException("Failed to retrieve the JobManager gateway.", e); + } + LOG.info("Checking and uploading JAR files"); try { JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout); @@ -432,6 +402,8 @@ public JobSubmissionResult runDetached(JobGraph jobGraph, ClassLoader classLoade * @throws Exception In case an error occurred. */ public void cancel(JobID jobId) throws Exception { + ActorGateway jobManagerGateway = getJobManagerGateway(); + Future response; try { response = jobManagerGateway.ask(new JobManagerMessages.CancelJob(jobId), timeout); @@ -471,6 +443,7 @@ public Map getAccumulators(JobID jobID) throws Exception { * @return A Map containing the accumulator's name and its value. */ public Map getAccumulators(JobID jobID, ClassLoader loader) throws Exception { + ActorGateway jobManagerGateway = getJobManagerGateway(); Future response; try { @@ -520,6 +493,8 @@ public void endSessions(List jobIds) throws Exception { if (jobIds == null) { throw new IllegalArgumentException("The JobIDs must not be null"); } + + ActorGateway jobManagerGateway = getJobManagerGateway(); for (JobID jid : jobIds) { if (jid != null) { @@ -572,4 +547,27 @@ private static JobGraph getJobGraph(FlinkPlan optPlan, List jarFiles, List< return job; } + // ------------------------------------------------------------------------ + // Helper methods + // ------------------------------------------------------------------------ + + /** + * Returns the {@link ActorGateway} of the current job manager leader using + * the {@link LeaderRetrievalService}. + * + * @return ActorGateway of the current job manager leader + * @throws Exception + */ + private ActorGateway getJobManagerGateway() throws Exception { + LOG.info("Looking up JobManager"); + LeaderRetrievalService leaderRetrievalService; + + leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config); + + return LeaderRetrievalUtils.retrieveLeaderGateway( + leaderRetrievalService, + actorSystem, + lookupTimeout); + } + } diff --git a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java index 424f72e834287..fb5200b9192f6 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/RemoteExecutorHostnameResolutionTest.java @@ -21,10 +21,10 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.junit.Test; -import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.URL; @@ -51,7 +51,7 @@ public void testUnresolvableHostname1() { exec.executePlan(getProgram()); fail("This should fail with an ProgramInvocationException"); } - catch (IOException e) { + catch (ProgramInvocationException e) { // that is what we want! assertTrue(e.getCause() instanceof UnknownHostException); } @@ -74,7 +74,7 @@ public void testUnresolvableHostname2() { exec.executePlan(getProgram()); fail("This should fail with an ProgramInvocationException"); } - catch (IOException e) { + catch (ProgramInvocationException e) { // that is what we want! assertTrue(e.getCause() instanceof UnknownHostException); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index a43688199bb4a..0105632380444 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -28,12 +28,12 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobClientMessages; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.util.SerializedThrowable; @@ -52,7 +52,6 @@ import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; -import java.net.UnknownHostException; import java.util.concurrent.TimeoutException; import static com.google.common.base.Preconditions.checkNotNull; @@ -84,36 +83,6 @@ public static ActorSystem startJobClientActorSystem(Configuration config) return system; } - /** - * Extracts the JobManager's Akka URL from the configuration. If localActorSystem is true, then - * the JobClient is executed in the same actor system as the JobManager. Thus, they can - * communicate locally. - * - * @param config Configuration object containing all user provided configuration values - * @return The socket address of the JobManager actor system - */ - public static InetSocketAddress getJobManagerAddress(Configuration config) throws IOException { - String jobManagerAddress = config.getString( - ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null); - - int jobManagerRPCPort = config.getInteger( - ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, - ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT); - - if (jobManagerAddress == null) { - throw new RuntimeException( - "JobManager address has not been specified in the configuration."); - } - - try { - return new InetSocketAddress( - InetAddress.getByName(jobManagerAddress), jobManagerRPCPort); - } - catch (UnknownHostException e) { - throw new IOException("Cannot resolve JobManager hostname " + jobManagerAddress, e); - } - } - /** * Sends a [[JobGraph]] to the JobClient actor specified by jobClient which submits it then to * the JobManager. The method blocks until the job has finished or the JobManager is no longer @@ -121,7 +90,8 @@ public static InetSocketAddress getJobManagerAddress(Configuration config) throw * case a [[JobExecutionException]] is thrown. * * @param actorSystem The actor system that performs the communication. - * @param jobManagerGateway Gateway to the JobManager that should execute the job. + * @param leaderRetrievalService Leader retrieval service which used to find the current leading + * JobManager * @param jobGraph JobGraph describing the Flink job * @param timeout Timeout for futures * @param sysoutLogUpdates prints log updates to system out if true @@ -131,14 +101,14 @@ public static InetSocketAddress getJobManagerAddress(Configuration config) throw */ public static JobExecutionResult submitJobAndWait( ActorSystem actorSystem, - ActorGateway jobManagerGateway, + LeaderRetrievalService leaderRetrievalService, JobGraph jobGraph, FiniteDuration timeout, boolean sysoutLogUpdates, ClassLoader classLoader) throws JobExecutionException { checkNotNull(actorSystem, "The actorSystem must not be null."); - checkNotNull(jobManagerGateway, "The jobManagerGateway must not be null."); + checkNotNull(leaderRetrievalService, "The jobManagerGateway must not be null."); checkNotNull(jobGraph, "The jobGraph must not be null."); checkNotNull(timeout, "The timeout must not be null."); @@ -146,12 +116,10 @@ public static JobExecutionResult submitJobAndWait( // the JobManager. It forwards the job submission, checks the success/failure responses, logs // update messages, watches for disconnect between client and JobManager, ... - Props jobClientActorProps = Props.create( - JobClientActor.class, - jobManagerGateway.actor(), - LOG, - sysoutLogUpdates, - jobManagerGateway.leaderSessionID()); + Props jobClientActorProps = JobClientActor.createJobClientActorProps( + leaderRetrievalService, + timeout, + sysoutLogUpdates); ActorRef jobClientActor = actorSystem.actorOf(jobClientActorProps); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java index bf747c46c1073..d08046be34341 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActor.java @@ -20,45 +20,93 @@ import akka.actor.ActorRef; import akka.actor.PoisonPill; +import akka.actor.Props; import akka.actor.Status; import akka.actor.Terminated; +import akka.dispatch.Futures; +import akka.dispatch.OnSuccess; import com.google.common.base.Preconditions; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.FlinkUntypedActor; import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.ExecutionGraphMessages; import org.apache.flink.runtime.messages.JobClientMessages; +import org.apache.flink.runtime.messages.JobClientMessages.JobManagerActorRef; +import org.apache.flink.runtime.messages.JobClientMessages.JobManagerLeaderAddress; +import org.apache.flink.runtime.messages.JobClientMessages.SubmitJobAndWait; import org.apache.flink.runtime.messages.JobManagerMessages; -import org.slf4j.Logger; +import org.apache.flink.runtime.util.SerializedThrowable; +import scala.concurrent.duration.FiniteDuration; +import java.io.IOException; import java.util.UUID; +import java.util.concurrent.Callable; /** * Actor which constitutes the bridge between the non-actor code and the JobManager. The JobClient * is used to submit jobs to the JobManager and to request the port of the BlobManager. */ -public class JobClientActor extends FlinkUntypedActor { - - private final ActorRef jobManager; - private final Logger logger; +public class JobClientActor extends FlinkUntypedActor implements LeaderRetrievalListener { + + private final LeaderRetrievalService leaderRetrievalService; + + /** timeout for futures */ + private final FiniteDuration timeout; + + /** true if status messages shall be printed to sysout */ private final boolean sysoutUpdates; + /** true if a SubmitJobSuccess message has been received */ + private boolean jobSuccessfullySubmitted = false; + + /** true if a PoisonPill was taken */ + private boolean terminated = false; + + /** ActorRef to the current leader */ + private ActorRef jobManager; + /** leader session ID of the JobManager when this actor was created */ - private final UUID leaderSessionID; + private UUID leaderSessionID; /** Actor which submits a job to the JobManager via this actor */ private ActorRef submitter; - public JobClientActor(ActorRef jobManager, Logger logger, boolean sysoutUpdates, - UUID leaderSessionID) { + /** JobGraph which shall be submitted to the JobManager */ + private JobGraph jobGraph; - this.jobManager = Preconditions.checkNotNull(jobManager, "The JobManager ActorRef must not be null."); - this.logger = Preconditions.checkNotNull(logger, "The logger must not be null."); - - this.leaderSessionID = leaderSessionID; + public JobClientActor( + LeaderRetrievalService leaderRetrievalService, + FiniteDuration submissionTimeout, + boolean sysoutUpdates) { + this.leaderRetrievalService = Preconditions.checkNotNull(leaderRetrievalService); + this.timeout = Preconditions.checkNotNull(submissionTimeout); this.sysoutUpdates = sysoutUpdates; } - + + @Override + public void preStart() { + try { + leaderRetrievalService.start(this); + } catch (Exception e) { + LOG.error("Could not start the leader retrieval service."); + throw new RuntimeException("Could not start the leader retrieval service.", e); + } + } + + @Override + public void postStop() { + try { + leaderRetrievalService.stop(); + } catch (Exception e) { + LOG.warn("Could not properly stop the leader retrieval service."); + } + } + @Override protected void handleMessage(Object message) { @@ -66,50 +114,79 @@ protected void handleMessage(Object message) { if (message instanceof ExecutionGraphMessages.ExecutionStateChanged) { logAndPrintMessage(message); - } - else if (message instanceof ExecutionGraphMessages.JobStatusChanged) { + } else if (message instanceof ExecutionGraphMessages.JobStatusChanged) { logAndPrintMessage(message); } + // ============ JobManager ActorRef resolution =============== + + else if (message instanceof JobManagerLeaderAddress) { + JobManagerLeaderAddress msg = (JobManagerLeaderAddress) message; + + disconnectFromJobManager(); + + this.leaderSessionID = msg.leaderSessionID(); + + if (msg.address() != null) { + // Resolve the job manager leader address to obtain an ActorRef + AkkaUtils.getActorRefFuture(msg.address(), getContext().system(), timeout) + .onSuccess(new OnSuccess() { + @Override + public void onSuccess(ActorRef result) throws Throwable { + getSelf().tell(decorateMessage(new JobManagerActorRef(result)), ActorRef.noSender()); + } + }, getContext().dispatcher()); + } + } else if (message instanceof JobManagerActorRef) { + // Resolved JobManager ActorRef + JobManagerActorRef msg = (JobManagerActorRef) message; + connectToJobManager(msg.jobManager()); + + if (jobGraph != null && !jobSuccessfullySubmitted) { + // if we haven't yet submitted the job successfully + tryToSubmitJob(jobGraph); + } + } + // =========== Job Life Cycle Messages =============== // submit a job to the JobManager - else if (message instanceof JobClientMessages.SubmitJobAndWait) { - // sanity check that this no job was submitted through this actor before - - // it is a one-shot actor after all - if (this.submitter == null) { - JobGraph jobGraph = ((JobClientMessages.SubmitJobAndWait) message).jobGraph(); - if (jobGraph == null) { - logger.error("Received null JobGraph"); - sender().tell( + else if (message instanceof SubmitJobAndWait) { + // only accept SubmitJobWait messages if we're not about to terminate + if (!terminated) { + // sanity check that this no job was submitted through this actor before - + // it is a one-shot actor after all + if (this.submitter == null) { + jobGraph = ((SubmitJobAndWait) message).jobGraph(); + if (jobGraph == null) { + LOG.error("Received null JobGraph"); + sender().tell( decorateMessage(new Status.Failure(new Exception("JobGraph is null"))), getSelf()); - } - else { - logger.info("Sending message to JobManager {} to submit job {} ({}) and wait for progress", - jobManager.path().toString(), jobGraph.getName(), jobGraph.getJobID()); + } else { + LOG.info("Received job {} ({}).", jobGraph.getName(), jobGraph.getJobID()); - this.submitter = getSender(); - jobManager.tell( - decorateMessage( - new JobManagerMessages.SubmitJob( - jobGraph, - ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES)), - getSelf()); - - // make sure we notify the sender when the connection got lost - getContext().watch(jobManager); + this.submitter = getSender(); + + // is only successful if we already know the job manager leader + tryToSubmitJob(jobGraph); + } + } else { + // repeated submission - tell failure to sender and kill self + String msg = "Received repeated 'SubmitJobAndWait'"; + LOG.error(msg); + getSender().tell( + decorateMessage(new Status.Failure(new Exception(msg))), ActorRef.noSender()); + + terminate(); } - } - else { - // repeated submission - tell failure to sender and kill self - String msg = "Received repeated 'SubmitJobAndWait'"; - logger.error(msg); + } else { + // we're about to receive a PoisonPill because terminated == true + String msg = getClass().getName() + " is about to be terminated. Therefore, the " + + "job submission cannot be executed."; + LOG.error(msg); getSender().tell( decorateMessage(new Status.Failure(new Exception(msg))), ActorRef.noSender()); - - getContext().unwatch(jobManager); - getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender()); } } // acknowledgement to submit job is only logged, our original @@ -117,41 +194,80 @@ else if (message instanceof JobClientMessages.SubmitJobAndWait) { else if (message instanceof JobManagerMessages.JobResultSuccess || message instanceof JobManagerMessages.JobResultFailure) { - if (logger.isDebugEnabled()) { - logger.debug("Received {} message from JobManager", message.getClass().getSimpleName()); + if (LOG.isDebugEnabled()) { + LOG.debug("Received {} message from JobManager", message.getClass().getSimpleName()); } // forward the success to the original job submitter - if (this.submitter != null) { + if (hasJobBeenSubmitted()) { this.submitter.tell(decorateMessage(message), getSelf()); } - - // we are done, stop ourselves - getContext().unwatch(jobManager); - getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender()); + + terminate(); } else if (message instanceof JobManagerMessages.JobSubmitSuccess) { // job was successfully submitted :-) - logger.info("Job was successfully submitted to the JobManager"); + LOG.info("Job was successfully submitted to the JobManager {}.", getSender().path()); + jobSuccessfullySubmitted = true; } - // =========== Actor / Communication Failure =============== + // =========== Actor / Communication Failure / Timeouts =============== else if (message instanceof Terminated) { ActorRef target = ((Terminated) message).getActor(); if (jobManager.equals(target)) { - String msg = "Lost connection to JobManager " + jobManager.path(); - logger.info(msg); - submitter.tell(decorateMessage(new Status.Failure(new Exception(msg))), getSelf()); + LOG.info("Lost connection to JobManager {}. Triggering connection timeout.", + jobManager.path()); + disconnectFromJobManager(); + + // we only issue a connection timeout if we have submitted a job before + // otherwise, we might have some more time to find another job manager + // Important: The ConnectionTimeout message is filtered out in case that we are + // notified about a new leader by setting the new leader session ID, because + // ConnectionTimeout extends RequiresLeaderSessionID + if (hasJobBeenSubmitted()) { + getContext().system().scheduler().scheduleOnce( + timeout, + getSelf(), + decorateMessage(JobClientMessages.getConnectionTimeout()), + getContext().dispatcher(), + ActorRef.noSender()); + } } else { - logger.error("Received 'Terminated' for unknown actor " + target); + LOG.warn("Received 'Terminated' for unknown actor " + target); + } + } else if (JobClientMessages.getConnectionTimeout().equals(message)) { + // check if we haven't found a job manager yet + if (!isConnected()) { + if (hasJobBeenSubmitted()) { + submitter.tell( + decorateMessage(new Status.Failure( + new JobClientActorConnectionTimeoutException("Lost connection to the JobManager."))), + getSelf()); + } + // Connection timeout reached, let's terminate + terminate(); + } + } else if (JobClientMessages.getSubmissionTimeout().equals(message)) { + // check if our job submission was successful in the meantime + if (!jobSuccessfullySubmitted) { + if (hasJobBeenSubmitted()) { + submitter.tell( + decorateMessage(new Status.Failure( + new JobClientActorSubmissionTimeoutException("Job submission to the JobManager timed out."))), + getSelf()); + } + + // We haven't heard back from the job manager after sending the job graph to him, + // therefore terminate + terminate(); } } // =========== Unknown Messages =============== else { - logger.error("JobClient received unknown message: " + message); + LOG.error("JobClient received unknown message: " + message); } } @@ -161,9 +277,133 @@ protected UUID getLeaderSessionID() { } private void logAndPrintMessage(Object message) { - logger.info(message.toString()); + LOG.info(message.toString()); if (sysoutUpdates) { System.out.println(message.toString()); } } + + @Override + public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + getSelf().tell( + decorateMessage(new JobManagerLeaderAddress(leaderAddress, leaderSessionID)), + getSelf()); + } + + @Override + public void handleError(Exception exception) { + LOG.error("Error occurred in the LeaderRetrievalService.", exception); + getSelf().tell(decorateMessage(PoisonPill.getInstance()), getSelf()); + } + + private void disconnectFromJobManager() { + if (jobManager != ActorRef.noSender()) { + getContext().unwatch(jobManager); + jobManager = ActorRef.noSender(); + } + } + + private void connectToJobManager(ActorRef jobManager) { + if (jobManager != ActorRef.noSender()) { + getContext().unwatch(jobManager); + } + + LOG.info("Connected to new JobManager {}.", jobManager.path()); + + this.jobManager = jobManager; + getContext().watch(jobManager); + } + + private void tryToSubmitJob(final JobGraph jobGraph) { + this.jobGraph = jobGraph; + + if (isConnected()) { + LOG.info("Sending message to JobManager {} to submit job {} ({}) and wait for progress", + jobManager.path().toString(), jobGraph.getName(), jobGraph.getJobID()); + + Futures.future(new Callable() { + @Override + public Object call() throws Exception { + ActorGateway jobManagerGateway = new AkkaActorGateway(jobManager, leaderSessionID); + + LOG.info("Upload jar files to job manager {}.", jobManager.path()); + + try { + JobClient.uploadJarFiles(jobGraph, jobManagerGateway, timeout); + } catch (IOException exception) { + getSelf().tell( + decorateMessage(new JobManagerMessages.JobResultFailure( + new SerializedThrowable( + new JobSubmissionException( + jobGraph.getJobID(), + "Could not upload the jar files to the job manager.", + exception) + ) + )), + ActorRef.noSender() + ); + } + + LOG.info("Submit job to the job manager {}.", jobManager.path()); + + jobManager.tell( + decorateMessage( + new JobManagerMessages.SubmitJob( + jobGraph, + ListeningBehaviour.EXECUTION_RESULT_AND_STATE_CHANGES)), + getSelf()); + + // issue a SubmissionTimeout message to check that we submit the job within + // the given timeout + getContext().system().scheduler().scheduleOnce( + timeout, + getSelf(), + decorateMessage(JobClientMessages.getSubmissionTimeout()), + getContext().dispatcher(), + ActorRef.noSender()); + + return null; + } + }, getContext().dispatcher()); + } else { + LOG.info("Could not submit job {} ({}), because there is no connection to a " + + "JobManager.", + jobGraph.getName(), jobGraph.getJobID()); + + // We want to submit a job, but we haven't found a job manager yet. + // Let's give him another chance to find a job manager within the given timeout. + getContext().system().scheduler().scheduleOnce( + timeout, + getSelf(), + decorateMessage(JobClientMessages.getConnectionTimeout()), + getContext().dispatcher(), + ActorRef.noSender() + ); + } + } + + private void terminate() { + terminated = true; + disconnectFromJobManager(); + getSelf().tell(decorateMessage(PoisonPill.getInstance()), ActorRef.noSender()); + } + + private boolean isConnected() { + return jobManager != ActorRef.noSender(); + } + + private boolean hasJobBeenSubmitted() { + return submitter != ActorRef.noSender(); + } + + public static Props createJobClientActorProps( + LeaderRetrievalService leaderRetrievalService, + FiniteDuration timeout, + boolean sysoutUpdates) { + return Props.create( + JobClientActor.class, + leaderRetrievalService, + timeout, + sysoutUpdates); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java new file mode 100644 index 0000000000000..72a56585f98ca --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java @@ -0,0 +1,35 @@ +/* + * 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.client; + +/** + * Exception which is thrown when the {@link JobClientActor} wants to submit a job to + * the job manager but has not found one after a given timeout interval. + */ +public class JobClientActorConnectionTimeoutException extends Exception { + private static final long serialVersionUID = 2287747430528388637L; + + public JobClientActorConnectionTimeoutException(String msg) { + super(msg); + } + + public JobClientActorConnectionTimeoutException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java new file mode 100644 index 0000000000000..2d394621abd53 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java @@ -0,0 +1,35 @@ +/* + * 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.client; + +/** + * Exception which is thrown by the {@link JobClientActor} if it has not heard back from the job + * manager after it has submitted a job to it within a given timeout interval. + */ +public class JobClientActorSubmissionTimeoutException extends Exception { + private static final long serialVersionUID = 8762463142030454853L; + + public JobClientActorSubmissionTimeoutException(String msg) { + super(msg); + } + + public JobClientActorSubmissionTimeoutException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index d2fe7f52981de..fe049cf91004c 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -371,6 +371,8 @@ class JobManager( akka.serialization.JavaSerializer.currentSystem.withValue( context.system.asInstanceOf[ExtendedActorSystem]) { + log.info(s"Recovering all jobs.") + val jobGraphs = submittedJobGraphs.recoverJobGraphs().asScala if (!leaderElectionService.hasLeadership()) { diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala index ac374939528a7..a60fa7aa093ce 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobClientMessages.scala @@ -18,8 +18,10 @@ package org.apache.flink.runtime.messages +import java.util.UUID + +import akka.actor.ActorRef import org.apache.flink.runtime.jobgraph.JobGraph -import org.apache.flink.runtime.util.SerializedThrowable /** * This object contains the [[org.apache.flink.runtime.client.JobClient]] specific messages @@ -47,4 +49,26 @@ object JobClientMessages { * @param jobGraph The job to be executed. */ case class SubmitJobDetached(jobGraph: JobGraph) + + /** Notifies the JobClientActor about a new leader address and a leader session ID. + * + * @param address New leader address + * @param leaderSessionID New leader session ID + */ + case class JobManagerLeaderAddress(address: String, leaderSessionID: UUID) + + /** Notifies the JobClientActor about the ActorRef of the new leader. + * + * @param jobManager ActorRef of the new leader + */ + case class JobManagerActorRef(jobManager: ActorRef) extends RequiresLeaderSessionID + + /** Message which is triggered when the submission timeout has been reached. */ + case object SubmissionTimeout extends RequiresLeaderSessionID + + /** Messaeg which is triggered when the connection timeout has been reached. */ + case object ConnectionTimeout extends RequiresLeaderSessionID + + def getSubmissionTimeout(): AnyRef = SubmissionTimeout + def getConnectionTimeout(): AnyRef = ConnectionTimeout } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala index 8097bdce89ff4..8a4d27b90758f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobManagerMessages.scala @@ -84,7 +84,7 @@ object JobManagerMessages { /** * Triggers recovery of all available jobs. */ - case class RecoverAllJobs() extends RequiresLeaderSessionID + case object RecoverAllJobs extends RequiresLeaderSessionID /** * Cancels a job with the given [[jobID]] at the JobManager. The result of the cancellation is @@ -427,4 +427,8 @@ object JobManagerMessages { def getRequestArchive: AnyRef = { RequestArchive } + + def getRecoverAllJobs: AnyRef = { + RecoverAllJobs + } } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index b795ecdb6efda..d80df83e60c5b 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -39,7 +39,7 @@ import org.apache.flink.runtime.jobmanager.{JobManager, RecoveryMode} import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalService, LeaderRetrievalListener, StandaloneLeaderRetrievalService} import org.apache.flink.runtime.messages.TaskManagerMessages.NotifyWhenRegisteredAtAnyJobManager -import org.apache.flink.runtime.util.{StandaloneUtils, ZooKeeperUtils} +import org.apache.flink.runtime.util.{LeaderRetrievalUtils, StandaloneUtils, ZooKeeperUtils} import org.apache.flink.runtime.webmonitor.WebMonitor import org.slf4j.LoggerFactory @@ -398,29 +398,29 @@ abstract class FlinkMiniCluster( : JobExecutionResult = { submitJobAndWait(jobGraph, printUpdates, timeout) } + + def submitJobAndWait( + jobGraph: JobGraph, + printUpdates: Boolean, + timeout: FiniteDuration) + : JobExecutionResult = { + submitJobAndWait(jobGraph, printUpdates, timeout, createLeaderRetrievalService()) + } @throws(classOf[JobExecutionException]) def submitJobAndWait( jobGraph: JobGraph, printUpdates: Boolean, - timeout: FiniteDuration) + timeout: FiniteDuration, + leaderRetrievalService: LeaderRetrievalService) : JobExecutionResult = { val clientActorSystem = startJobClientActorSystem(jobGraph.getJobID) try { - val jobManagerGateway = try { - getLeaderGateway(timeout) - } catch { - case e: Exception => throw new JobExecutionException( - jobGraph.getJobID, - "Could not retrieve leading job manager gateway.", - e) - } - JobClient.submitJobAndWait( clientActorSystem, - jobManagerGateway, + leaderRetrievalService, jobGraph, timeout, printUpdates, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index b28fb7326aff7..d9d9596c34a9f 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1117,7 +1117,12 @@ class TaskManager( currentJobManager match { case Some(jm) => - handleJobManagerDisconnect(jm, s"JobManager ${newJobManagerAkkaURL} was elected as leader.") + Option(newJobManagerAkkaURL) match { + case Some(newJMAkkaURL) => + handleJobManagerDisconnect(jm, s"JobManager ${newJMAkkaURL} was elected as leader.") + case None => + handleJobManagerDisconnect(jm, s"Old JobManager lost its leadership.") + } case None => } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java new file mode 100644 index 0000000000000..a93a51563a7c5 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorRecoveryITCase.java @@ -0,0 +1,163 @@ +/* + * 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.client; + +import akka.actor.PoisonPill; +import org.apache.curator.test.TestingServer; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.testingUtils.TestingCluster; +import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import scala.concurrent.Await; +import scala.concurrent.Promise; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.util.concurrent.TimeUnit; + + +public class JobClientActorRecoveryITCase extends TestLogger { + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + public static TestingServer zkServer; + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(); + + zkServer.start(); + } + + public static void teardown() throws Exception { + if (zkServer != null) { + zkServer.stop(); + zkServer = null; + } + } + + /** + * Tests wether the JobClientActor can connect to a newly elected leading job manager to obtain + * the JobExecutionResult. The submitted job blocks for the first execution attempt. The + * leading job manager will be killed so that the second job manager will be elected as the + * leader. The newly elected leader has to retrieve the checkpointed job from ZooKeeper + * and continue its execution. This time, the job does not block and, thus, can be finished. + * The execution result should be sent to the JobClientActor which originally submitted the + * job. + * + * @throws Exception + */ + @Test + public void testJobClientRecovery() throws Exception { + File rootFolder = tempFolder.getRoot(); + + Configuration config = ZooKeeperTestUtils.createZooKeeperRecoveryModeConfig( + zkServer.getConnectString(), + rootFolder.getPath()); + + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 2); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + + final TestingCluster cluster = new TestingCluster(config); + cluster.start(); + + JobVertex blockingVertex = new JobVertex("Blocking Vertex"); + blockingVertex.setInvokableClass(BlockingTask.class); + blockingVertex.setParallelism(1); + final JobGraph jobGraph = new JobGraph("Blocking Test Job", blockingVertex); + final Promise promise = new scala.concurrent.impl.Promise.DefaultPromise<>(); + + Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); + + try { + Thread submitter = new Thread(new Runnable() { + @Override + public void run() { + try { + JobExecutionResult result = cluster.submitJobAndWait(jobGraph, false); + promise.success(result); + } catch (Exception e) { + promise.failure(e); + } + } + }); + + submitter.start(); + + synchronized (BlockingTask.waitLock) { + while (BlockingTask.HasBlockedExecution < 1 && deadline.hasTimeLeft()) { + BlockingTask.waitLock.wait(deadline.timeLeft().toMillis()); + } + } + + if (deadline.isOverdue()) { + Assert.fail("The job has not blocked within the given deadline."); + } + + ActorGateway gateway = cluster.getLeaderGateway(deadline.timeLeft()); + + gateway.tell(TestingJobManagerMessages.getDisablePostStop()); + gateway.tell(PoisonPill.getInstance()); + + // if the job fails then an exception is thrown here + Await.result(promise.future(), deadline.timeLeft()); + } finally { + cluster.shutdown(); + } + } + + public static class BlockingTask extends AbstractInvokable { + + private volatile static int BlockExecution = 1; + private volatile static int HasBlockedExecution = 0; + private static Object waitLock = new Object(); + + @Override + public void registerInputOutput() throws Exception { + // Nothing to do + } + + @Override + public void invoke() throws Exception { + if (BlockExecution > 0) { + BlockExecution--; + + // Tell the test that it's OK to kill the leader + synchronized (waitLock) { + HasBlockedExecution++; + waitLock.notifyAll(); + } + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java new file mode 100644 index 0000000000000..00ad632b343e7 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobClientActorTest.java @@ -0,0 +1,228 @@ +/* + * 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.client; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.PoisonPill; +import akka.actor.Props; +import akka.pattern.Patterns; +import akka.testkit.JavaTestKit; +import akka.util.Timeout; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.FlinkUntypedActor; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.messages.JobClientMessages; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.messages.Messages; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +public class JobClientActorTest extends TestLogger { + + private static ActorSystem system; + private static JobGraph testJobGraph = new JobGraph("Test Job"); + + @BeforeClass + public static void setup() { + system = AkkaUtils.createLocalActorSystem(new Configuration()); + } + + @AfterClass + public static void teardown() { + JavaTestKit.shutdownActorSystem(system); + system = null; + } + + /** Tests that a {@link JobClientActorSubmissionTimeoutException} is thrown when the job cannot + * be submitted by the JobClientActor. This is here the case, because the started JobManager + * never replies to a SubmitJob message. + * + * @throws Exception + */ + @Test(expected=JobClientActorSubmissionTimeoutException.class) + public void testSubmissionTimeout() throws Exception { + FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS); + FiniteDuration timeout = jobClientActorTimeout.$times(2); + + UUID leaderSessionID = UUID.randomUUID(); + + ActorRef jobManager = system.actorOf( + Props.create( + PlainActor.class, + leaderSessionID)); + + TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService( + jobManager.path().toString(), + leaderSessionID + ); + + Props jobClientActorProps = JobClientActor.createJobClientActorProps( + testingLeaderRetrievalService, + jobClientActorTimeout, + false); + + ActorRef jobClientActor = system.actorOf(jobClientActorProps); + + + Future jobExecutionResult = Patterns.ask( + jobClientActor, + new JobClientMessages.SubmitJobAndWait(testJobGraph), + new Timeout(timeout)); + + Await.result(jobExecutionResult, timeout); + } + + /** Tests that a {@link org.apache.flink.runtime.client.JobClientActorConnectionTimeoutException} + * is thrown when the JobClientActor wants to submit a job but has not connected to a JobManager. + * + * @throws Exception + */ + @Test(expected=JobClientActorConnectionTimeoutException.class) + public void testConnectionTimeoutWithoutJobManager() throws Exception { + FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS); + FiniteDuration timeout = jobClientActorTimeout.$times(2); + + TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService(); + + Props jobClientActorProps = JobClientActor.createJobClientActorProps( + testingLeaderRetrievalService, + jobClientActorTimeout, + false); + + ActorRef jobClientActor = system.actorOf(jobClientActorProps); + + Future jobExecutionResult = Patterns.ask( + jobClientActor, + new JobClientMessages.SubmitJobAndWait(testJobGraph), + new Timeout(timeout)); + + Await.result(jobExecutionResult, timeout); + } + + /** Tests that a {@link org.apache.flink.runtime.client.JobClientActorConnectionTimeoutException} + * is thrown after a successful job submission if the JobManager dies. + * + * @throws Exception + */ + @Test(expected=JobClientActorConnectionTimeoutException.class) + public void testConnectionTimeoutAfterJobSubmission() throws Exception { + FiniteDuration jobClientActorTimeout = new FiniteDuration(5, TimeUnit.SECONDS); + FiniteDuration timeout = jobClientActorTimeout.$times(2); + + UUID leaderSessionID = UUID.randomUUID(); + + ActorRef jobManager = system.actorOf( + Props.create( + JobAcceptingActor.class, + leaderSessionID)); + + TestingLeaderRetrievalService testingLeaderRetrievalService = new TestingLeaderRetrievalService( + jobManager.path().toString(), + leaderSessionID + ); + + Props jobClientActorProps = JobClientActor.createJobClientActorProps( + testingLeaderRetrievalService, + jobClientActorTimeout, + false); + + ActorRef jobClientActor = system.actorOf(jobClientActorProps); + + Future jobExecutionResult = Patterns.ask( + jobClientActor, + new JobClientMessages.SubmitJobAndWait(testJobGraph), + new Timeout(timeout)); + + Future waitFuture = Patterns.ask(jobManager, new RegisterTest(), new Timeout(timeout)); + + Await.result(waitFuture, timeout); + + jobManager.tell(PoisonPill.getInstance(), ActorRef.noSender()); + + Await.result(jobExecutionResult, timeout); + } + + public static class PlainActor extends FlinkUntypedActor { + + private final UUID leaderSessionID; + + public PlainActor(UUID leaderSessionID) { + this.leaderSessionID = leaderSessionID; + } + + @Override + protected void handleMessage(Object message) throws Exception { + + } + + @Override + protected UUID getLeaderSessionID() { + return leaderSessionID; + } + } + + public static class JobAcceptingActor extends FlinkUntypedActor { + private final UUID leaderSessionID; + private boolean jobAccepted = false; + private ActorRef testFuture = ActorRef.noSender(); + + public JobAcceptingActor(UUID leaderSessionID) { + this.leaderSessionID = leaderSessionID; + } + + @Override + protected void handleMessage(Object message) throws Exception { + if (message instanceof JobManagerMessages.SubmitJob) { + getSender().tell( + new JobManagerMessages.JobSubmitSuccess(((JobManagerMessages.SubmitJob) message).jobGraph().getJobID()), + getSelf()); + + jobAccepted = true; + + if(testFuture != ActorRef.noSender()) { + testFuture.tell(Messages.getAcknowledge(), getSelf()); + } + } else if (message instanceof RegisterTest) { + testFuture = getSender(); + + if (jobAccepted) { + testFuture.tell(Messages.getAcknowledge(), getSelf()); + } + } + } + + @Override + protected UUID getLeaderSessionID() { + return leaderSessionID; + } + } + + public static class RegisterTest{} +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index 5753cdef7af35..aa03fe1dd5174 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -18,10 +18,8 @@ package org.apache.flink.runtime.io.network.partition; -import akka.actor.ActorSystem; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -46,7 +44,6 @@ public class PartialConsumePipelinedResultTest { private final static int NUMBER_OF_NETWORK_BUFFERS = 128; private static TestingCluster flink; - private static ActorSystem jobClient; @BeforeClass public static void setUp() throws Exception { @@ -59,8 +56,6 @@ public static void setUp() throws Exception { flink = new TestingCluster(config, true); flink.start(); - - jobClient = JobClient.startJobClientActorSystem(flink.configuration()); } @AfterClass @@ -102,13 +97,7 @@ public void testPartialConsumePipelinedResultReceiver() throws Exception { sender.setSlotSharingGroup(slotSharingGroup); receiver.setSlotSharingGroup(slotSharingGroup); - JobClient.submitJobAndWait( - jobClient, - flink.getLeaderGateway(TestingUtils.TESTING_DURATION()), - jobGraph, - TestingUtils.TESTING_DURATION(), - false, - this.getClass().getClassLoader()); + flink.submitJobAndWait(jobGraph, false, TestingUtils.TESTING_DURATION()); } // --------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java index bbd8fadcd3ad2..c804830a1d970 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java @@ -43,11 +43,14 @@ import org.apache.flink.runtime.testingUtils.TestingJobManager; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import scala.concurrent.Await; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -56,6 +59,9 @@ public class JobManagerLeaderElectionTest extends TestLogger { + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + private static ActorSystem actorSystem; private static TestingServer testingServer; private static Timeout timeout = new Timeout(TestingUtils.TESTING_DURATION()); @@ -84,12 +90,10 @@ public static void teardown() throws Exception { */ @Test public void testLeaderElection() throws Exception { - final Configuration configuration = new Configuration(); - - configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper"); - configuration.setString( - ConfigConstants.ZOOKEEPER_QUORUM_KEY, - testingServer.getConnectString()); + final Configuration configuration = ZooKeeperTestUtils + .createZooKeeperRecoveryModeConfig( + testingServer.getConnectString(), + tempFolder.getRoot().getPath()); ActorRef jm = null; @@ -115,12 +119,11 @@ public void testLeaderElection() throws Exception { */ @Test public void testLeaderReelection() throws Exception { - final Configuration configuration = new Configuration(); + final Configuration configuration = ZooKeeperTestUtils + .createZooKeeperRecoveryModeConfig( + testingServer.getConnectString(), + tempFolder.getRoot().getPath()); - configuration.setString(ConfigConstants.RECOVERY_MODE, "zookeeper"); - configuration.setString( - ConfigConstants.ZOOKEEPER_QUORUM_KEY, - testingServer.getConnectString()); ActorRef jm; ActorRef jm2 = null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java index 8dd380e9e9d6e..0b84474231a27 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeStateCleanupTest.java @@ -129,7 +129,7 @@ public void testStateCleanupAfterNewLeaderElectionAndListenerNotification() thro // try to resubmit now the non-blocking job, it should complete successfully Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false); - cluster.submitJobAndWait(job, false, timeout); + cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(jm2.path(), jm2.leaderSessionID())); } /** @@ -207,7 +207,7 @@ public void testReelectionOfSameJobManager() throws Exception { UUID leaderSessionID = UUID.randomUUID(); UUID newLeaderSessionID = UUID.randomUUID(); - FiniteDuration shortTimeout = new FiniteDuration(20, TimeUnit.SECONDS); + FiniteDuration shortTimeout = new FiniteDuration(10, TimeUnit.SECONDS); cluster.grantLeadership(0, leaderSessionID); cluster.notifyRetrievalListeners(0, leaderSessionID); @@ -244,10 +244,11 @@ public void testReelectionOfSameJobManager() throws Exception { cluster.waitForTaskManagersToBeRegistered(); + ActorGateway leaderGateway = cluster.getLeaderGateway(timeout); + // try to resubmit now the non-blocking job, it should complete successfully Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false); - cluster.submitJobAndWait(job, false, timeout); - + cluster.submitJobAndWait(job, false, timeout, new TestingLeaderRetrievalService(leaderGateway.path(), leaderGateway.leaderSessionID())); } public JobGraph createBlockingJob(int parallelism) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java index 5b63107d7d6cd..c83f5486099a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java @@ -25,6 +25,8 @@ import org.apache.flink.runtime.testingUtils.TestingCluster; import scala.Option; +import java.util.ArrayList; +import java.util.List; import java.util.UUID; /** @@ -39,11 +41,8 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster { private final boolean useSingleActorSystem; private final StreamingMode streamingMode; - public TestingLeaderElectionService[] leaderElectionServices; - public TestingLeaderRetrievalService[] leaderRetrievalServices; - - private int leaderElectionServiceCounter = 0; - private int leaderRetrievalServiceCounter = 0; + public List leaderElectionServices; + public List leaderRetrievalServices; private int leaderIndex = -1; @@ -58,8 +57,8 @@ public LeaderElectionRetrievalTestingCluster( this.useSingleActorSystem = singleActorSystem; this.streamingMode = streamingMode; - leaderElectionServices = new TestingLeaderElectionService[this.numJobManagers()]; - leaderRetrievalServices = new TestingLeaderRetrievalService[this.numTaskManagers() + 1]; + leaderElectionServices = new ArrayList(); + leaderRetrievalServices = new ArrayList(); } @Override @@ -79,18 +78,18 @@ public boolean useSingleActorSystem() { @Override public Option createLeaderElectionService() { - leaderElectionServices[leaderElectionServiceCounter] = new TestingLeaderElectionService(); + leaderElectionServices.add(new TestingLeaderElectionService()); - LeaderElectionService result = leaderElectionServices[leaderElectionServiceCounter++]; + LeaderElectionService result = leaderElectionServices.get(leaderElectionServices.size() - 1); return Option.apply(result); } @Override public LeaderRetrievalService createLeaderRetrievalService() { - leaderRetrievalServices[leaderRetrievalServiceCounter] = new TestingLeaderRetrievalService(); + leaderRetrievalServices.add(new TestingLeaderRetrievalService()); - return leaderRetrievalServices[leaderRetrievalServiceCounter++]; + return leaderRetrievalServices.get(leaderRetrievalServices.size() - 1); } @Override @@ -103,11 +102,11 @@ public int getNumberOfJobManagers() { public void grantLeadership(int index, UUID leaderSessionID) { if(leaderIndex >= 0) { // first revoke leadership - leaderElectionServices[leaderIndex].notLeader(); + leaderElectionServices.get(leaderIndex).notLeader(); } // make the JM with index the new leader - leaderElectionServices[index].isLeader(leaderSessionID); + leaderElectionServices.get(index).isLeader(leaderSessionID); leaderIndex = index; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java index ea058f4bc9836..4e119feebec15 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java @@ -18,16 +18,13 @@ package org.apache.flink.runtime.leaderelection; -import java.io.Serializable; import java.util.UUID; /** * Test {@link LeaderElectionService} implementation which directly forwards isLeader and notLeader * calls to the contender. */ -public class TestingLeaderElectionService implements LeaderElectionService, Serializable { - - private static final long serialVersionUID = -8007939683948014574L; +public class TestingLeaderElectionService implements LeaderElectionService { private LeaderContender contender; private boolean hasLeadership = false; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java index 43902fdcfb63d..c44fc2a6904f3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderRetrievalService.java @@ -29,11 +29,27 @@ */ public class TestingLeaderRetrievalService implements LeaderRetrievalService { + private final String leaderAddress; + private final UUID leaderSessionID; + private LeaderRetrievalListener listener; + public TestingLeaderRetrievalService() { + this(null, null); + } + + public TestingLeaderRetrievalService(String leaderAddress, UUID leaderSessionID) { + this.leaderAddress = leaderAddress; + this.leaderSessionID = leaderSessionID; + } + @Override public void start(LeaderRetrievalListener listener) throws Exception { this.listener = listener; + + if (leaderAddress != null) { + listener.notifyLeaderAddress(leaderAddress, leaderSessionID); + } } @Override diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala index e91f06815a894..b8f4ede2dbe0b 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala @@ -70,6 +70,18 @@ trait TestingJobManagerLike extends FlinkActor { var disconnectDisabled = false + var postStopEnabled = true + + abstract override def postStop(): Unit = { + if (postStopEnabled) { + super.postStop() + } else { + // only stop leader election service to revoke the leadership of this JM so that a new JM + // can be elected leader + leaderElectionService.stop() + } + } + abstract override def handleMessage: Receive = { handleTestingMessage orElse super.handleMessage } @@ -270,6 +282,9 @@ trait TestingJobManagerLike extends FlinkActor { case DisableDisconnect => disconnectDisabled = true + case DisablePostStop => + postStopEnabled = false + case msg: Disconnect => if (!disconnectDisabled) { super.handleMessage(msg) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala index 4f5cf146a2ce3..e4d0a6f74bf0c 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala @@ -87,5 +87,12 @@ object TestingJobManagerMessages { */ case class NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager: Int) - def getNotifyWhenLeader: AnyRef = NotifyWhenLeader + /** Disables the post stop method of the [[TestingJobManager]]. + * + * Only the leaderElectionService is stopped in the postStop method call to revoke the leadership + */ + case object DisablePostStop + + def getNotifyWhenLeader(): AnyRef = NotifyWhenLeader + def getDisablePostStop(): AnyRef = DisablePostStop } diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index ed2113a1b781c..5c7a93277622f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -22,19 +22,20 @@ import akka.actor.Kill; import akka.actor.PoisonPill; import org.apache.commons.io.FileUtils; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobClient; -import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.Tasks; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; @@ -42,9 +43,13 @@ import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.concurrent.Await; import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; import scala.concurrent.duration.FiniteDuration; +import scala.concurrent.impl.Promise; import java.io.File; import java.io.IOException; @@ -128,7 +133,7 @@ public void testTaskManagerRegistrationAtReelectedLeader() throws Exception { @Test public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { int numJMs = 10; - int numTMs = 3; + int numTMs = 2; int numSlotsPerTM = 3; int parallelism = numTMs * numSlotsPerTM; @@ -141,10 +146,9 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem"); configuration.setString(ConfigConstants.STATE_BACKEND_FS_RECOVERY_PATH, tempDirectory.getPath()); - // @TODO @tillrohrmann temporary "disable" recovery, because currently the client does - // not need to resubmit a failed job to a new leader. Should we keep this test and - // disable recovery fully or will this be subsumed by the real client changes anyways? - configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, timeout.toString()); + // we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make + // sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message + configuration.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, AkkaUtils.INF_TIMEOUT().toString()); Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(true); @@ -186,51 +190,55 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { thread.start(); + Deadline deadline = timeout.$times(3).fromNow(); + // Kill all JobManager except for two - for (int i = 0; i < numJMs - 2; i++) { - ActorGateway jm = cluster.getLeaderGateway(timeout); + for (int i = 0; i < numJMs; i++) { + ActorGateway jm = cluster.getLeaderGateway(deadline.timeLeft()); cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor()); - Future future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout); - - Await.ready(future, timeout); + // recover all jobs, sent manually + log.info("Sent recover all jobs manually to job manager {}.", jm.path()); + jm.tell(JobManagerMessages.getRecoverAllJobs()); - cluster.clearLeader(); - - jm.tell(Kill.getInstance()); - } + if (i < numJMs - 1) { + Future future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), deadline.timeLeft()); - ActorGateway jm = cluster.getLeaderGateway(timeout); + Await.ready(future, deadline.timeLeft()); - cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor()); + cluster.clearLeader(); - Future future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), timeout); + if (i == numJMs - 2) { + Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false); + } - Await.ready(future, timeout); + log.info("Kill job manager {}.", jm.path()); - cluster.clearLeader(); + jm.tell(TestingJobManagerMessages.getDisablePostStop()); + jm.tell(Kill.getInstance()); + } + } - // set the BlockinOnceReceiver for the execution on the last JM to non-blocking, so - // that it can succeed - Tasks.BlockingOnceReceiver$.MODULE$.blocking_$eq(false); + log.info("Waiting for submitter thread to terminate."); - jm.tell(PoisonPill.getInstance()); + thread.join(deadline.timeLeft().toMillis()); - thread.join(timeout.toMillis()); + log.info("Submitter thread has terminated."); if (thread.isAlive()) { - jobSubmission.finished = true; fail("The job submission thread did not stop (meaning it did not succeeded in" + "executing the test job."); } + + Await.result(jobSubmission.resultPromise.future(), deadline.timeLeft()); } finally { if (clientActorSystem != null) { cluster.shutdownJobClientActorSystem(clientActorSystem); } - if (thread != null && thread.isAlive() && jobSubmission != null) { + if (thread != null && thread.isAlive()) { jobSubmission.finished = true; } cluster.stop(); @@ -238,12 +246,15 @@ public void testJobExecutionOnClusterWithLeaderReelection() throws Exception { } public static class JobSubmitterRunnable implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(JobSubmitterRunnable.class); boolean finished = false; final ActorSystem clientActorSystem; final ForkableFlinkMiniCluster cluster; final JobGraph graph; + final Promise resultPromise = new Promise.DefaultPromise<>(); + public JobSubmitterRunnable( ActorSystem actorSystem, ForkableFlinkMiniCluster cluster, @@ -255,39 +266,23 @@ public JobSubmitterRunnable( @Override public void run() { - while (!finished) { - try { - LeaderRetrievalService lrService = - LeaderRetrievalUtils.createLeaderRetrievalService( - cluster.configuration()); - - ActorGateway jobManagerGateway = - LeaderRetrievalUtils.retrieveLeaderGateway( - lrService, - clientActorSystem, - timeout); - - JobClient.submitJobAndWait( - clientActorSystem, - jobManagerGateway, - graph, - timeout, - false, - getClass().getClassLoader()); - - finished = true; - } - catch (JobExecutionException e) { - // This was expected, so just try again to submit the job - } - catch (LeaderRetrievalException e) { - // This can also happen, so just try again to submit the job - } - catch (Exception e) { - // This was not expected... fail the test case - e.printStackTrace(); - fail("Caught unexpected exception in job submission test case."); - } + try { + LeaderRetrievalService lrService = + LeaderRetrievalUtils.createLeaderRetrievalService( + cluster.configuration()); + + JobExecutionResult result = JobClient.submitJobAndWait( + clientActorSystem, + lrService, + graph, + timeout, + false, + getClass().getClassLoader()); + + resultPromise.success(result); + } catch (Exception e) { + // This was not expected... fail the test case + resultPromise.failure(e); } } } diff --git a/flink-tests/src/test/resources/log4j-test.properties b/flink-tests/src/test/resources/log4j-test.properties index 9d298411e0608..3ba6d1d11c15e 100644 --- a/flink-tests/src/test/resources/log4j-test.properties +++ b/flink-tests/src/test/resources/log4j-test.properties @@ -18,7 +18,8 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -log4j.rootLogger=OFF, testlogger +log4j.rootLogger=INFO, testlogger +log4j.logger.org.apache.flink.runtime.client.JobClientActor=DEBUG # A1 is set to be a ConsoleAppender. log4j.appender.testlogger=org.apache.log4j.ConsoleAppender @@ -27,5 +28,5 @@ log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n # suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger -log4j.logger.org.apache.zookeeper=OFF, testlogger \ No newline at end of file +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR +log4j.logger.org.apache.zookeeper=OFF \ No newline at end of file