You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2016/10/13 15:40:11 UTC
[1/8] flink git commit: [FLINK-4375] [distributed coordination]
Implement new JobManager creation, initialization, and basic RPC methods
Repository: flink
Updated Branches:
refs/heads/flip-6 35a44daa6 -> 48c936eed
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index a2716e5..9f9234f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -18,13 +18,13 @@
package org.apache.flink.runtime.taskexecutor;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.runtime.accumulators.AccumulatorSnapshot;
import org.apache.flink.runtime.blob.BlobCache;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.instance.InstanceID;
import org.apache.flink.runtime.concurrent.ApplyFunction;
import org.apache.flink.runtime.concurrent.Future;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
@@ -33,20 +33,29 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.PartitionInfo;
import org.apache.flink.runtime.filecache.FileCache;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.memory.MemoryManager;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.metrics.MetricRegistry;
-import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered;
-import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup;
import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRegistered;
+import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestRejected;
import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.TMSlotRequestReply;
import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
import org.apache.flink.runtime.taskexecutor.exceptions.CheckpointException;
import org.apache.flink.runtime.taskexecutor.exceptions.PartitionException;
import org.apache.flink.runtime.taskexecutor.exceptions.TaskException;
@@ -60,26 +69,16 @@ import org.apache.flink.runtime.taskmanager.Task;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import org.apache.flink.runtime.taskmanager.TaskManagerActions;
import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.RpcService;
-
import org.apache.flink.util.Preconditions;
-import java.util.HashSet;
-import java.util.Set;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
+import java.util.Set;
import java.util.UUID;
import static org.apache.flink.util.Preconditions.checkArgument;
@@ -276,11 +275,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
TaskMetricGroup taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd);
InputSplitProvider inputSplitProvider = new RpcInputSplitProvider(
- jobManagerConnection.getJobManagerGateway(),
- tdd.getJobID(),
- tdd.getVertexID(),
- tdd.getExecutionId(),
- taskManagerConfiguration.getTimeout());
+ jobManagerConnection.getJobMasterLeaderId(),
+ jobManagerConnection.getJobManagerGateway(),
+ tdd.getJobID(),
+ tdd.getVertexID(),
+ tdd.getExecutionId(),
+ taskManagerConfiguration.getTimeout());
TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions();
CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder();
@@ -580,10 +580,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
clearTasks();
}
- private void updateTaskExecutionState(final JobMasterGateway jobMasterGateway, final TaskExecutionState taskExecutionState) {
+ private void updateTaskExecutionState(
+ final UUID jobMasterLeaderId,
+ final JobMasterGateway jobMasterGateway,
+ final TaskExecutionState taskExecutionState)
+ {
final ExecutionAttemptID executionAttemptID = taskExecutionState.getID();
- Future<Acknowledge> futureAcknowledge = jobMasterGateway.updateTaskExecutionState(taskExecutionState);
+ Future<Acknowledge> futureAcknowledge = jobMasterGateway.updateTaskExecutionState(
+ jobMasterLeaderId, taskExecutionState);
futureAcknowledge.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
@Override
@@ -595,7 +600,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
}, getMainThreadExecutor());
}
- private void unregisterTaskAndNotifyFinalState(final JobMasterGateway jobMasterGateway, ExecutionAttemptID executionAttemptID) {
+ private void unregisterTaskAndNotifyFinalState(
+ final UUID jobMasterLeaderId,
+ final JobMasterGateway jobMasterGateway,
+ final ExecutionAttemptID executionAttemptID)
+ {
Task task = removeTask(executionAttemptID);
if (task != null) {
@@ -613,13 +622,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot();
updateTaskExecutionState(
- jobMasterGateway,
- new TaskExecutionState(
- task.getJobID(),
- task.getExecutionId(),
- task.getExecutionState(),
- task.getFailureCause(),
- accumulatorSnapshot));
+ jobMasterLeaderId,
+ jobMasterGateway,
+ new TaskExecutionState(
+ task.getJobID(),
+ task.getExecutionId(),
+ task.getExecutionState(),
+ task.getFailureCause(),
+ accumulatorSnapshot));
} else {
log.error("Cannot find task with ID {} to unregister.", executionAttemptID);
}
@@ -661,11 +671,14 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
}
}
- private JobManagerConnection associateWithJobManager(JobMasterGateway jobMasterGateway, int blobPort) {
+ private JobManagerConnection associateWithJobManager(UUID jobMasterLeaderId,
+ JobMasterGateway jobMasterGateway, int blobPort)
+ {
+ Preconditions.checkNotNull(jobMasterLeaderId);
Preconditions.checkNotNull(jobMasterGateway);
Preconditions.checkArgument(blobPort > 0 || blobPort <= 65535, "Blob port is out of range.");
- TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterGateway);
+ TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterLeaderId, jobMasterGateway);
CheckpointResponder checkpointResponder = new RpcCheckpointResponder(jobMasterGateway);
@@ -678,19 +691,21 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
taskManagerConfiguration.getCleanupInterval());
ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier(
- jobMasterGateway,
- getRpcService().getExecutor(),
- taskManagerConfiguration.getTimeout());
+ jobMasterLeaderId,
+ jobMasterGateway,
+ getRpcService().getExecutor(),
+ taskManagerConfiguration.getTimeout());
- PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway);
+ PartitionStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterLeaderId, jobMasterGateway);
return new JobManagerConnection(
- jobMasterGateway,
- taskManagerActions,
- checkpointResponder,
- libraryCacheManager,
- resultPartitionConsumableNotifier,
- partitionStateChecker);
+ jobMasterLeaderId,
+ jobMasterGateway,
+ taskManagerActions,
+ checkpointResponder,
+ libraryCacheManager,
+ resultPartitionConsumableNotifier,
+ partitionStateChecker);
}
private void disassociateFromJobManager(JobManagerConnection jobManagerConnection) throws IOException {
@@ -782,9 +797,11 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
}
private class TaskManagerActionsImpl implements TaskManagerActions {
+ private final UUID jobMasterLeaderId;
private final JobMasterGateway jobMasterGateway;
- private TaskManagerActionsImpl(JobMasterGateway jobMasterGateway) {
+ private TaskManagerActionsImpl(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) {
+ this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
}
@@ -793,7 +810,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
runAsync(new Runnable() {
@Override
public void run() {
- unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID);
+ unregisterTaskAndNotifyFinalState(jobMasterLeaderId, jobMasterGateway, executionAttemptID);
}
});
}
@@ -816,7 +833,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
@Override
public void updateTaskExecutionState(final TaskExecutionState taskExecutionState) {
- TaskExecutor.this.updateTaskExecutionState(jobMasterGateway, taskExecutionState);
+ TaskExecutor.this.updateTaskExecutionState(jobMasterLeaderId, jobMasterGateway, taskExecutionState);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
index 4850d63..3b9da48 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java
@@ -31,7 +31,10 @@ import org.apache.flink.runtime.jobmaster.SerializedInputSplit;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
+import java.util.UUID;
+
public class RpcInputSplitProvider implements InputSplitProvider {
+ private final UUID jobMasterLeaderId;
private final JobMasterGateway jobMasterGateway;
private final JobID jobID;
private final JobVertexID jobVertexID;
@@ -39,11 +42,13 @@ public class RpcInputSplitProvider implements InputSplitProvider {
private final Time timeout;
public RpcInputSplitProvider(
+ UUID jobMasterLeaderId,
JobMasterGateway jobMasterGateway,
JobID jobID,
JobVertexID jobVertexID,
ExecutionAttemptID executionAttemptID,
Time timeout) {
+ this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
this.jobID = Preconditions.checkNotNull(jobID);
this.jobVertexID = Preconditions.checkNotNull(jobVertexID);
@@ -56,7 +61,8 @@ public class RpcInputSplitProvider implements InputSplitProvider {
public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws InputSplitProviderException {
Preconditions.checkNotNull(userCodeClassLoader);
- Future<SerializedInputSplit> futureInputSplit = jobMasterGateway.requestNextInputSplit(jobVertexID, executionAttemptID);
+ Future<SerializedInputSplit> futureInputSplit = jobMasterGateway.requestNextInputSplit(
+ jobMasterLeaderId, jobVertexID, executionAttemptID);
try {
SerializedInputSplit serializedInputSplit = futureInputSplit.get(timeout.getSize(), timeout.getUnit());
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
index ab111ad..1c91b87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java
@@ -28,11 +28,15 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobmaster.JobMasterGateway;
import org.apache.flink.util.Preconditions;
+import java.util.UUID;
+
public class RpcPartitionStateChecker implements PartitionStateChecker {
+ private final UUID jobMasterLeaderId;
private final JobMasterGateway jobMasterGateway;
- public RpcPartitionStateChecker(JobMasterGateway jobMasterGateway) {
+ public RpcPartitionStateChecker(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) {
+ this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
}
@@ -43,6 +47,6 @@ public class RpcPartitionStateChecker implements PartitionStateChecker {
IntermediateDataSetID resultId,
ResultPartitionID partitionId) {
- return jobMasterGateway.requestPartitionState(partitionId, executionId, resultId);
+ return jobMasterGateway.requestPartitionState(jobMasterLeaderId, partitionId, executionId, resultId);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
index 29ad3b6..cf01d5a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java
@@ -31,27 +31,32 @@ import org.apache.flink.util.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.UUID;
import java.util.concurrent.Executor;
public class RpcResultPartitionConsumableNotifier implements ResultPartitionConsumableNotifier {
private static final Logger LOG = LoggerFactory.getLogger(RpcResultPartitionConsumableNotifier.class);
+ private final UUID jobMasterLeaderId;
private final JobMasterGateway jobMasterGateway;
private final Executor executor;
private final Time timeout;
public RpcResultPartitionConsumableNotifier(
+ UUID jobMasterLeaderId,
JobMasterGateway jobMasterGateway,
Executor executor,
Time timeout) {
+ this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
this.executor = Preconditions.checkNotNull(executor);
this.timeout = Preconditions.checkNotNull(timeout);
}
@Override
public void notifyPartitionConsumable(JobID jobId, ResultPartitionID partitionId, final TaskActions taskActions) {
- Future<Acknowledge> acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(partitionId, timeout);
+ Future<Acknowledge> acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(
+ jobMasterLeaderId, partitionId, timeout);
acknowledgeFuture.exceptionallyAsync(new ApplyFunction<Throwable, Void>() {
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
index 84f5ac7..9209d15 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
@@ -581,4 +581,10 @@ object AkkaUtils {
throw new Exception(s"Could not retrieve InetSocketAddress from Akka URL $akkaURL")
}
}
+
+ def formatDurationParingErrorMessage: String = {
+ "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " +
+ "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|"+
+ "(�s|micro|microsecond)|(ns|nano|nanosecond)"
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
index faf69cc..a255027 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
@@ -19,11 +19,15 @@
package org.apache.flink.runtime.highavailability;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.VoidBlobStore;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.nonha.NonHaRegistry;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import java.io.IOException;
import java.util.concurrent.ConcurrentHashMap;
/**
@@ -140,4 +144,14 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
}
}
+
+ @Override
+ public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+ return new NonHaRegistry();
+ }
+
+ @Override
+ public BlobStore createBlobStore() throws IOException {
+ return new VoidBlobStore();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
index 30dfef5..f709cbd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java
@@ -21,14 +21,21 @@ package org.apache.flink.runtime.jobmaster;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobStore;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;
import org.junit.After;
import org.junit.Before;
+import org.junit.Ignore;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.mockito.PowerMockito;
@@ -61,11 +68,19 @@ public class JobManagerRunnerMockTest {
private TestingOnCompletionActions jobCompletion;
+ private BlobStore blobStore;
+
+ private RunningJobsRegistry runningJobsRegistry;
+
@Before
public void setUp() throws Exception {
+ RpcService mockRpc = mock(RpcService.class);
+ when(mockRpc.getAddress()).thenReturn("localhost");
+
jobManager = mock(JobMaster.class);
jobManagerGateway = mock(JobMasterGateway.class);
when(jobManager.getSelf()).thenReturn(jobManagerGateway);
+ when(jobManager.getRpcService()).thenReturn(mockRpc);
PowerMockito.whenNew(JobMaster.class).withAnyArguments().thenReturn(jobManager);
@@ -74,19 +89,25 @@ public class JobManagerRunnerMockTest {
leaderElectionService = mock(LeaderElectionService.class);
when(leaderElectionService.hasLeadership()).thenReturn(true);
- submittedJobGraphStore = mock(SubmittedJobGraphStore.class);
- when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(true);
+ runningJobsRegistry = mock(RunningJobsRegistry.class);
+ when(runningJobsRegistry.isJobRunning(any(JobID.class))).thenReturn(true);
+ blobStore = mock(BlobStore.class);
+
HighAvailabilityServices haServices = mock(HighAvailabilityServices.class);
when(haServices.getJobManagerLeaderElectionService(any(JobID.class))).thenReturn(leaderElectionService);
when(haServices.getSubmittedJobGraphStore()).thenReturn(submittedJobGraphStore);
+ when(haServices.createBlobStore()).thenReturn(blobStore);
+ when(haServices.getRunningJobsRegistry()).thenReturn(runningJobsRegistry);
runner = PowerMockito.spy(new JobManagerRunner(
- new JobGraph("test"),
+ new JobGraph("test", new JobVertex("vertex")),
mock(Configuration.class),
- mock(RpcService.class),
+ mockRpc,
haServices,
- mock(JobManagerServices.class),
+ JobManagerServices.fromConfiguration(new Configuration(), haServices),
+ new MetricRegistry(MetricRegistryConfiguration.defaultMetricRegistryConfiguration()),
+ jobCompletion,
jobCompletion));
}
@@ -94,25 +115,26 @@ public class JobManagerRunnerMockTest {
public void tearDown() throws Exception {
}
+ @Ignore
@Test
public void testStartAndShutdown() throws Exception {
runner.start();
- verify(jobManager).init();
- verify(jobManager).start();
verify(leaderElectionService).start(runner);
assertTrue(!jobCompletion.isJobFinished());
assertTrue(!jobCompletion.isJobFailed());
+ verify(jobManager).start(any(UUID.class));
+
runner.shutdown();
verify(leaderElectionService).stop();
verify(jobManager).shutDown();
}
+ @Ignore
@Test
public void testShutdownBeforeGrantLeadership() throws Exception {
runner.start();
- verify(jobManager).init();
verify(jobManager).start();
verify(leaderElectionService).start(runner);
@@ -129,13 +151,14 @@ public class JobManagerRunnerMockTest {
}
+ @Ignore
@Test
public void testJobFinished() throws Exception {
runner.start();
UUID leaderSessionID = UUID.randomUUID();
runner.grantLeadership(leaderSessionID);
- verify(jobManagerGateway).startJob(leaderSessionID);
+ verify(jobManager).start(leaderSessionID);
assertTrue(!jobCompletion.isJobFinished());
// runner been told by JobManager that job is finished
@@ -148,13 +171,14 @@ public class JobManagerRunnerMockTest {
assertTrue(runner.isShutdown());
}
+ @Ignore
@Test
public void testJobFailed() throws Exception {
runner.start();
UUID leaderSessionID = UUID.randomUUID();
runner.grantLeadership(leaderSessionID);
- verify(jobManagerGateway).startJob(leaderSessionID);
+ verify(jobManager).start(leaderSessionID);
assertTrue(!jobCompletion.isJobFinished());
// runner been told by JobManager that job is failed
@@ -166,39 +190,41 @@ public class JobManagerRunnerMockTest {
assertTrue(runner.isShutdown());
}
+ @Ignore
@Test
public void testLeadershipRevoked() throws Exception {
runner.start();
UUID leaderSessionID = UUID.randomUUID();
runner.grantLeadership(leaderSessionID);
- verify(jobManagerGateway).startJob(leaderSessionID);
+ verify(jobManager).start(leaderSessionID);
assertTrue(!jobCompletion.isJobFinished());
runner.revokeLeadership();
- verify(jobManagerGateway).suspendJob(any(Throwable.class));
+ verify(jobManager).suspendExecution(any(Throwable.class));
assertFalse(runner.isShutdown());
}
+ @Ignore
@Test
public void testRegainLeadership() throws Exception {
runner.start();
UUID leaderSessionID = UUID.randomUUID();
runner.grantLeadership(leaderSessionID);
- verify(jobManagerGateway).startJob(leaderSessionID);
+ verify(jobManager).start(leaderSessionID);
assertTrue(!jobCompletion.isJobFinished());
runner.revokeLeadership();
- verify(jobManagerGateway).suspendJob(any(Throwable.class));
+ verify(jobManager).suspendExecution(any(Throwable.class));
assertFalse(runner.isShutdown());
UUID leaderSessionID2 = UUID.randomUUID();
runner.grantLeadership(leaderSessionID2);
- verify(jobManagerGateway).startJob(leaderSessionID2);
+ verify(jobManager).start(leaderSessionID2);
}
- private static class TestingOnCompletionActions implements OnCompletionActions {
+ private static class TestingOnCompletionActions implements OnCompletionActions, FatalErrorHandler {
private volatile JobExecutionResult result;
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
new file mode 100644
index 0000000..174422f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java
@@ -0,0 +1,24 @@
+/*
+ * 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.jobmaster;
+
+public class JobManagerRunnerTest {
+
+ // TODO: Test that
+}
[3/8] flink git commit: [FLINK-4375] [distributed coordination]
Implement new JobManager creation, initialization, and basic RPC methods
Posted by se...@apache.org.
[FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/34fef475
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/34fef475
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/34fef475
Branch: refs/heads/flip-6
Commit: 34fef4752e94b3d0c7afe7a9525799bb651a07b4
Parents: c8dc074
Author: Kurt Young <yk...@gmail.com>
Authored: Wed Oct 12 23:25:16 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200
----------------------------------------------------------------------
.../org/apache/flink/util/ExceptionUtils.java | 20 +
.../java/org/apache/flink/util/StringUtils.java | 14 +
.../apache/flink/runtime/blob/BlobServer.java | 65 +-
.../apache/flink/runtime/blob/BlobStore.java | 3 +-
.../apache/flink/runtime/blob/BlobUtils.java | 8 +-
.../flink/runtime/blob/FileSystemBlobStore.java | 53 +-
.../flink/runtime/blob/VoidBlobStore.java | 2 +-
.../CheckpointCoordinatorGateway.java | 26 +-
.../HighAvailabilityServices.java | 13 +
.../runtime/highavailability/NonHaServices.java | 13 +
.../highavailability/RunningJobsRegistry.java | 66 ++
.../highavailability/ZookeeperHaServices.java | 115 ++-
.../highavailability/nonha/NonHaRegistry.java | 62 ++
.../runtime/jobmanager/OnCompletionActions.java | 3 +-
.../runtime/jobmaster/JobManagerRunner.java | 270 ++++--
.../runtime/jobmaster/JobManagerServices.java | 93 +-
.../flink/runtime/jobmaster/JobMaster.java | 902 +++++++------------
.../runtime/jobmaster/JobMasterGateway.java | 114 +--
.../jobmaster/MiniClusterJobDispatcher.java | 61 +-
.../message/DisposeSavepointResponse.java | 49 -
.../message/TriggerSavepointResponse.java | 74 --
.../apache/flink/runtime/rpc/RpcService.java | 4 +-
.../taskexecutor/JobManagerConnection.java | 25 +-
.../runtime/taskexecutor/TaskExecutor.java | 107 ++-
.../taskexecutor/rpc/RpcInputSplitProvider.java | 8 +-
.../rpc/RpcPartitionStateChecker.java | 8 +-
.../RpcResultPartitionConsumableNotifier.java | 7 +-
.../apache/flink/runtime/akka/AkkaUtils.scala | 6 +
.../TestingHighAvailabilityServices.java | 14 +
.../jobmaster/JobManagerRunnerMockTest.java | 58 +-
.../runtime/jobmaster/JobManagerRunnerTest.java | 24 +
31 files changed, 1278 insertions(+), 1009 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
index 7227006..0f6f24f 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -99,6 +99,26 @@ public final class ExceptionUtils {
}
/**
+ * Throws the given {@code Throwable} in scenarios where the signatures do allow to
+ * throw a Exception. Errors and Exceptions are thrown directly, other "exotic"
+ * subclasses of Throwable are wrapped in an Exception.
+ *
+ * @param t The throwable to be thrown.
+ * @param parentMessage The message for the parent Exception, if one is needed.
+ */
+ public static void rethrowException(Throwable t, String parentMessage) throws Exception {
+ if (t instanceof Error) {
+ throw (Error) t;
+ }
+ else if (t instanceof Exception) {
+ throw (Exception) t;
+ }
+ else {
+ throw new Exception(parentMessage, t);
+ }
+ }
+
+ /**
* Tries to throw the given {@code Throwable} in scenarios where the signatures allows only IOExceptions
* (and RuntimeException and Error). Throws this exception directly, if it is an IOException,
* a RuntimeException, or an Error. Otherwise does nothing.
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
index 10b6304..3c32d77 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
@@ -335,4 +335,18 @@ public final class StringUtils {
return null;
}
}
+
+ public static boolean isNullOrWhitespaceOnly(String str) {
+ if (str == null || str.length() == 0) {
+ return true;
+ }
+
+ final int len = str.length();
+ for (int i = 0; i < len; i++) {
+ if (!Character.isWhitespace(str.charAt(i))) {
+ return false;
+ }
+ }
+ return true;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.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 ff54b67..5395d34 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
@@ -22,7 +22,11 @@ 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.configuration.HighAvailabilityOptions;
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.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.flink.util.NetUtils;
import org.slf4j.Logger;
@@ -43,6 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
/**
* This class implements the BLOB server. The BLOB server is responsible for listening for incoming requests and
@@ -88,23 +93,22 @@ public class BlobServer extends Thread implements BlobService {
* thrown if the BLOB server cannot bind to a free network port
*/
public BlobServer(Configuration config) throws IOException {
- checkNotNull(config, "Configuration");
+ this(config, createBlobStoreFromConfig(config));
+ }
- HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
+ public BlobServer(Configuration config, HighAvailabilityServices haServices) throws IOException {
+ this(config, haServices.createBlobStore());
+ }
+
+ private BlobServer(Configuration config, BlobStore blobStore) throws IOException {
+ checkNotNull(config);
+ this.blobStore = checkNotNull(blobStore);
// 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);
- if (highAvailabilityMode == HighAvailabilityMode.NONE) {
- this.blobStore = new VoidBlobStore();
- } else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
- this.blobStore = new FileSystemBlobStore(config);
- } else {
- throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + ".");
- }
-
// configure the maximum number of concurrent connections
final int maxConnections = config.getInteger(
ConfigConstants.BLOB_FETCH_CONCURRENT_KEY, ConfigConstants.DEFAULT_BLOB_FETCH_CONCURRENT);
@@ -125,13 +129,7 @@ public class BlobServer extends Thread implements BlobService {
backlog = ConfigConstants.DEFAULT_BLOB_FETCH_BACKLOG;
}
- if (highAvailabilityMode == HighAvailabilityMode.NONE) {
- // Add shutdown hook to delete storage directory
- this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
- }
- else {
- this.shutdownHook = null;
- }
+ this.shutdownHook = BlobUtils.addShutdownHook(this, LOG);
// ----------------------- start the server -------------------
@@ -426,4 +424,37 @@ public class BlobServer extends Thread implements BlobService {
}
}
+ private static BlobStore createBlobStoreFromConfig(Configuration config) throws IOException {
+ HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(config);
+
+ if (highAvailabilityMode == HighAvailabilityMode.NONE) {
+ return new VoidBlobStore();
+ } else if (highAvailabilityMode == HighAvailabilityMode.ZOOKEEPER) {
+ final String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+ if (isNullOrWhitespaceOnly(storagePath)) {
+ throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
+ HighAvailabilityOptions.HA_STORAGE_PATH);
+ }
+
+ final Path path;
+ try {
+ path = new Path(storagePath);
+ } catch (Exception e) {
+ throw new IOException("Invalid path for highly available storage (" +
+ HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+ }
+
+ final FileSystem fileSystem;
+ try {
+ fileSystem = path.getFileSystem();
+ } catch (Exception e) {
+ throw new IOException("Could not create FileSystem for highly available storage (" +
+ HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+ }
+
+ return new FileSystemBlobStore(fileSystem, storagePath);
+ } else {
+ throw new IllegalConfigurationException("Unexpected high availability mode '" + highAvailabilityMode + ".");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java
----------------------------------------------------------------------
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
index 1e72d91..7050338 100644
--- 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
@@ -25,7 +25,7 @@ import java.io.File;
/**
* A blob store.
*/
-interface BlobStore {
+public interface BlobStore {
/**
* Copies the local file to the blob store.
@@ -93,5 +93,4 @@ interface BlobStore {
* Cleans up the store and deletes all blobs.
*/
void cleanUp();
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java
----------------------------------------------------------------------
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 e74fa6f..136df09 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
@@ -24,6 +24,7 @@ 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.util.IOUtils;
+import org.apache.flink.util.StringUtils;
import org.slf4j.Logger;
import java.io.EOFException;
@@ -73,7 +74,7 @@ public class BlobUtils {
*/
static File initStorageDirectory(String storageDirectory) {
File baseDir;
- if (storageDirectory == null || storageDirectory.trim().isEmpty()) {
+ if (StringUtils.isNullOrWhitespaceOnly(storageDirectory)) {
baseDir = new File(System.getProperty("java.io.tmpdir"));
}
else {
@@ -81,10 +82,9 @@ public class BlobUtils {
}
File storageDir;
- final int MAX_ATTEMPTS = 10;
- int attempt;
- for(attempt = 0; attempt < MAX_ATTEMPTS; attempt++) {
+ final int MAX_ATTEMPTS = 10;
+ for(int attempt = 0; attempt < MAX_ATTEMPTS; attempt++) {
storageDir = new File(baseDir, String.format(
"blobStore-%s", UUID.randomUUID().toString()));
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
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
index deba738..2c05002 100644
--- 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
@@ -20,12 +20,7 @@ package org.apache.flink.runtime.blob;
import com.google.common.io.Files;
-import org.apache.commons.lang3.StringUtils;
-
import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.util.IOUtils;
@@ -38,7 +33,6 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
-import java.net.URI;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -47,25 +41,24 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
*
* <p>This is used in addition to the local blob storage for high availability.
*/
-class FileSystemBlobStore implements BlobStore {
+public class FileSystemBlobStore implements BlobStore {
private static final Logger LOG = LoggerFactory.getLogger(FileSystemBlobStore.class);
+ /** The file system in which blobs are stored */
+ private final FileSystem fileSystem;
+
/** The base path of the blob store */
private final String basePath;
- FileSystemBlobStore(Configuration config) throws IOException {
- String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
-
- if (storagePath == null || StringUtils.isBlank(storagePath)) {
- throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
- " Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
- }
+ public FileSystemBlobStore(FileSystem fileSystem, String storagePath) throws IOException {
+ this.fileSystem = checkNotNull(fileSystem);
+ this.basePath = checkNotNull(storagePath) + "/blob";
- this.basePath = storagePath + "/blob";
+ LOG.info("Creating highly available BLOB storage directory at {}", basePath);
- FileSystem.get(new Path(basePath).toUri()).mkdirs(new Path(basePath));
- LOG.info("Created blob directory {}.", basePath);
+ fileSystem.mkdirs(new Path(basePath));
+ LOG.debug("Created highly available BLOB storage directory at {}", basePath);
}
// - Put ------------------------------------------------------------------
@@ -81,9 +74,7 @@ class FileSystemBlobStore implements BlobStore {
}
private void put(File fromFile, String toBlobPath) throws Exception {
- try (OutputStream os = FileSystem.get(new URI(toBlobPath))
- .create(new Path(toBlobPath), true)) {
-
+ try (OutputStream os = fileSystem.create(new Path(toBlobPath), true)) {
LOG.debug("Copying from {} to {}.", fromFile, toBlobPath);
Files.copy(fromFile, os);
}
@@ -106,16 +97,15 @@ class FileSystemBlobStore implements BlobStore {
checkNotNull(toFile, "File");
if (!toFile.exists() && !toFile.createNewFile()) {
- throw new IllegalStateException("Failed to create target file to copy to");
+ throw new IOException("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);
-
+ if (fileSystem.exists(fromPath)) {
+ try (InputStream is = fileSystem.open(fromPath);
+ FileOutputStream fos = new FileOutputStream(toFile))
+ {
LOG.debug("Copying from {} to {}.", fromBlobPath, toFile);
IOUtils.copyBytes(is, fos); // closes the streams
}
@@ -145,17 +135,16 @@ class FileSystemBlobStore implements BlobStore {
private void delete(String blobPath) {
try {
LOG.debug("Deleting {}.", blobPath);
-
- FileSystem fs = FileSystem.get(new URI(blobPath));
+
Path path = new Path(blobPath);
- fs.delete(path, true);
+ fileSystem.delete(path, true);
// send a call to delete the directory containing the file. This will
// fail (and be ignored) when some files still exist.
try {
- fs.delete(path.getParent(), false);
- fs.delete(new Path(basePath), false);
+ fileSystem.delete(path.getParent(), false);
+ fileSystem.delete(new Path(basePath), false);
} catch (IOException ignored) {}
}
catch (Exception e) {
@@ -168,7 +157,7 @@ class FileSystemBlobStore implements BlobStore {
try {
LOG.debug("Cleaning up {}.", basePath);
- FileSystem.get(new URI(basePath)).delete(new Path(basePath), true);
+ fileSystem.delete(new Path(basePath), true);
}
catch (Exception e) {
LOG.error("Failed to clean up recovery directory.");
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java
----------------------------------------------------------------------
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
index 1b71add..ece2ac1 100644
--- 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
@@ -25,7 +25,7 @@ import java.io.File;
/**
* A blob store doing nothing.
*/
-class VoidBlobStore implements BlobStore {
+public class VoidBlobStore implements BlobStore {
@Override
public void put(File localFile, BlobKey blobKey) throws Exception {
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
index e448ebc..196ef5c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
@@ -23,21 +23,23 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.state.CheckpointStateHandles;
+import java.util.UUID;
+
public interface CheckpointCoordinatorGateway extends RpcGateway {
void acknowledgeCheckpoint(
- JobID jobID,
- ExecutionAttemptID executionAttemptID,
- long checkpointID,
- CheckpointStateHandles checkpointStateHandles,
- long synchronousDurationMillis,
- long asynchronousDurationMillis,
- long bytesBufferedInAlignment,
- long alignmentDurationNanos);
+ final JobID jobID,
+ final ExecutionAttemptID executionAttemptID,
+ final long checkpointID,
+ final CheckpointStateHandles checkpointStateHandles,
+ final long synchronousDurationMillis,
+ final long asynchronousDurationMillis,
+ final long bytesBufferedInAlignment,
+ final long alignmentDurationNanos);
void declineCheckpoint(
- JobID jobID,
- ExecutionAttemptID executionAttemptID,
- long checkpointID,
- long checkpointTimestamp);
+ final JobID jobID,
+ final ExecutionAttemptID executionAttemptID,
+ final long checkpointID,
+ final long checkpointTimestamp);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
index a26886a..5d78ffc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
@@ -19,11 +19,14 @@
package org.apache.flink.runtime.highavailability;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.BlobStore;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import java.io.IOException;
+
/**
* This class gives access to all services needed for
*
@@ -72,4 +75,14 @@ public interface HighAvailabilityServices {
* Gets the submitted job graph store for the job manager
*/
SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception;
+
+ /**
+ * Gets the registry that holds information about whether jobs are currently running.
+ */
+ RunningJobsRegistry getRunningJobsRegistry() throws Exception;
+
+ /**
+ * Creates the BLOB store in which BLOBs are stored in a highly-available fashion.
+ */
+ BlobStore createBlobStore() throws IOException;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
index 2c6295c..d7fd2bf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
@@ -19,8 +19,11 @@
package org.apache.flink.runtime.highavailability;
import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.blob.BlobStore;
+import org.apache.flink.runtime.blob.VoidBlobStore;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
+import org.apache.flink.runtime.highavailability.nonha.NonHaRegistry;
import org.apache.flink.runtime.jobmanager.StandaloneSubmittedJobGraphStore;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
@@ -102,4 +105,14 @@ public class NonHaServices implements HighAvailabilityServices {
public SubmittedJobGraphStore getSubmittedJobGraphStore() throws Exception {
return new StandaloneSubmittedJobGraphStore();
}
+
+ @Override
+ public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+ return new NonHaRegistry();
+ }
+
+ @Override
+ public BlobStore createBlobStore() {
+ return new VoidBlobStore();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java
new file mode 100644
index 0000000..e7c131c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.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.highavailability;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.IOException;
+
+/**
+ * This registry tracks if a certain job is running.
+ *
+ * <p>This registry is used in highly-available setups with multiple master nodes,
+ * to determine whether a new leader should attempt to recover a certain job (because the
+ * job is still running), or whether the job has already finished successfully (in case of a
+ * finite job) and the leader has only been granted leadership because the previous leader
+ * quit cleanly after the job was finished.
+ */
+public interface RunningJobsRegistry {
+
+ /**
+ * Marks a job as running.
+ *
+ * @param jobID The id of the job.
+ *
+ * @throws IOException Thrown when the communication with the highly-available storage or registry
+ * failed and could not be retried.
+ */
+ void setJobRunning(JobID jobID) throws IOException;
+
+ /**
+ * Marks a job as running.
+ *
+ * @param jobID The id of the job.
+ *
+ * @throws IOException Thrown when the communication with the highly-available storage or registry
+ * failed and could not be retried.
+ */
+ void setJobFinished(JobID jobID) throws IOException;
+
+ /**
+ * Checks whether a job is running.
+ *
+ * @param jobID The id of the job to check.
+ * @return True if the job is still running, false otherwise.
+ *
+ * @throws IOException Thrown when the communication with the highly-available storage or registry
+ * failed and could not be retried.
+ */
+ boolean isJobRunning(JobID jobID) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
index d26b668..3a7736b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ZookeeperHaServices.java
@@ -19,8 +19,15 @@
package org.apache.flink.runtime.highavailability;
import org.apache.curator.framework.CuratorFramework;
+
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+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.blob.BlobStore;
+import org.apache.flink.runtime.blob.FileSystemBlobStore;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory;
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
@@ -28,12 +35,56 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;
+import java.io.IOException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
/**
- * An implementation of the {@link HighAvailabilityServices} with zookeeper.
+ * An implementation of the {@link HighAvailabilityServices} using Apache ZooKeeper.
+ * The services store data in ZooKeeper's nodes as illustrated by teh following tree structure:
+ *
+ * <pre>
+ * /flink
+ * +/cluster_id_1/resource_manager_lock
+ * | |
+ * | +/job-id-1/job_manager_lock
+ * | | /checkpoints/latest
+ * | | /latest-1
+ * | | /latest-2
+ * | |
+ * | +/job-id-2/job_manager_lock
+ * |
+ * +/cluster_id_2/resource_manager_lock
+ * |
+ * +/job-id-1/job_manager_lock
+ * |/checkpoints/latest
+ * | /latest-1
+ * |/persisted_job_graph
+ * </pre>
+ *
+ * <p>The root path "/flink" is configurable via the option {@link HighAvailabilityOptions#HA_ZOOKEEPER_ROOT}.
+ * This makes sure Flink stores its data under specific subtrees in ZooKeeper, for example to
+ * accommodate specific permission.
+ *
+ * <p>The "cluster_id" part identifies the data stored for a specific Flink "cluster".
+ * This "cluster" can be either a standalone or containerized Flink cluster, or it can be job
+ * on a framework like YARN or Mesos (in a "per-job-cluster" mode).
+ *
+ * <p>In case of a "per-job-cluster" on YARN or Mesos, the cluster-id is generated and configured
+ * automatically by the client or dispatcher that submits the Job to YARN or Mesos.
+ *
+ * <p>In the case of a standalone cluster, that cluster-id needs to be configured via
+ * {@link HighAvailabilityOptions#HA_CLUSTER_ID}. All nodes with the same cluster id will join the same
+ * cluster and participate in the execution of the same set of jobs.
*/
public class ZookeeperHaServices implements HighAvailabilityServices {
- private static final String DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX = "/resource-manager";
+ private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock";
+
+ private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock";
+
+ // ------------------------------------------------------------------------
/** The ZooKeeper client to use */
private final CuratorFramework client;
@@ -41,29 +92,37 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
/** The runtime configuration */
private final Configuration configuration;
+ public ZookeeperHaServices(Configuration configuration) {
+ this(ZooKeeperUtils.startCuratorFramework(configuration), configuration);
+ }
+
public ZookeeperHaServices(final CuratorFramework client, final Configuration configuration) {
- this.client = client;
- this.configuration = configuration;
+ this.client = checkNotNull(client);
+ this.configuration = checkNotNull(configuration);
}
+ // ------------------------------------------------------------------------
+ // Services
+ // ------------------------------------------------------------------------
+
@Override
public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
- return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+ return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
}
@Override
public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) throws Exception {
- return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathSuffixForJob(jobID));
+ return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID));
}
@Override
public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
- return ZooKeeperUtils.createLeaderElectionService(client, configuration, DEFAULT_RESOURCE_MANAGER_PATH_SUFFIX);
+ return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH);
}
@Override
public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) throws Exception {
- return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathSuffixForJob(jobID));
+ return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID));
}
@Override
@@ -76,7 +135,43 @@ public class ZookeeperHaServices implements HighAvailabilityServices {
return ZooKeeperUtils.createSubmittedJobGraphs(client, configuration);
}
- private static String getPathSuffixForJob(final JobID jobID) {
- return String.format("/job-managers/%s", jobID);
+ @Override
+ public RunningJobsRegistry getRunningJobsRegistry() throws Exception {
+ throw new UnsupportedOperationException("not yet implemented");
+ }
+
+ @Override
+ public BlobStore createBlobStore() throws IOException {
+ final String storagePath = configuration.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+ if (isNullOrWhitespaceOnly(storagePath)) {
+ throw new IllegalConfigurationException("Configuration is missing the mandatory parameter: " +
+ HighAvailabilityOptions.HA_STORAGE_PATH);
+ }
+
+ final Path path;
+ try {
+ path = new Path(storagePath);
+ } catch (Exception e) {
+ throw new IOException("Invalid path for highly available storage (" +
+ HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+ }
+
+ final FileSystem fileSystem;
+ try {
+ fileSystem = path.getFileSystem();
+ } catch (Exception e) {
+ throw new IOException("Could not create FileSystem for highly available storage (" +
+ HighAvailabilityOptions.HA_STORAGE_PATH.key() + ')', e);
+ }
+
+ return new FileSystemBlobStore(fileSystem, storagePath);
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static String getPathForJobManager(final JobID jobID) {
+ return "/" + jobID + JOB_MANAGER_LEADER_PATH;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
new file mode 100644
index 0000000..85dd711
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/NonHaRegistry.java
@@ -0,0 +1,62 @@
+/*
+ * 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.highavailability.nonha;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
+
+import java.util.HashSet;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A registry for running jobs, not-highly available.
+ */
+public class NonHaRegistry implements RunningJobsRegistry {
+
+ /** The currently running jobs */
+ private final HashSet<JobID> running = new HashSet<>();
+
+ @Override
+ public void setJobRunning(JobID jobID) {
+ checkNotNull(jobID);
+
+ synchronized (running) {
+ running.add(jobID);
+ }
+ }
+
+ @Override
+ public void setJobFinished(JobID jobID) {
+ checkNotNull(jobID);
+
+ synchronized (running) {
+ running.remove(jobID);
+ }
+ }
+
+ @Override
+ public boolean isJobRunning(JobID jobID) {
+ checkNotNull(jobID);
+
+ synchronized (running) {
+ return running.contains(jobID);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
index 6de4253..25a2a66 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/OnCompletionActions.java
@@ -19,9 +19,8 @@
package org.apache.flink.runtime.jobmanager;
import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-public interface OnCompletionActions extends FatalErrorHandler {
+public interface OnCompletionActions {
void jobFinished(JobExecutionResult result);
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
index a096932..74c1050 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java
@@ -21,26 +21,38 @@ package org.apache.flink.runtime.jobmaster;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
-import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
import org.apache.flink.runtime.leaderelection.LeaderContender;
import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.MetricRegistryConfiguration;
+import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.IOException;
import java.util.UUID;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
/**
* The runner for the job manager. It deals with job level leader election and make underlying job manager
* properly reacted.
*/
-public class JobManagerRunner implements LeaderContender, OnCompletionActions {
+public class JobManagerRunner implements LeaderContender, OnCompletionActions, FatalErrorHandler {
- private final Logger log = LoggerFactory.getLogger(JobManagerRunner.class);
+ private static final Logger log = LoggerFactory.getLogger(JobManagerRunner.class);
+
+ // ------------------------------------------------------------------------
/** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously */
private final Object lock = new Object();
@@ -48,52 +60,141 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
/** The job graph needs to run */
private final JobGraph jobGraph;
- private final OnCompletionActions toNotify;
+ /** The listener to notify once the job completes - either successfully or unsuccessfully */
+ private final OnCompletionActions toNotifyOnComplete;
+
+ /** The handler to call in case of fatal (unrecoverable) errors */
+ private final FatalErrorHandler errorHandler;
/** Used to check whether a job needs to be run */
- private final SubmittedJobGraphStore submittedJobGraphStore;
+ private final RunningJobsRegistry runningJobsRegistry;
/** Leader election for this job */
private final LeaderElectionService leaderElectionService;
+ private final JobManagerServices jobManagerServices;
+
private final JobMaster jobManager;
+ private final JobManagerMetricGroup jobManagerMetricGroup;
+
/** flag marking the runner as shut down */
private volatile boolean shutdown;
+ // ------------------------------------------------------------------------
+
public JobManagerRunner(
- final JobGraph jobGraph,
- final Configuration configuration,
- final RpcService rpcService,
- final HighAvailabilityServices haServices,
- final OnCompletionActions toNotify) throws Exception
+ final JobGraph jobGraph,
+ final Configuration configuration,
+ final RpcService rpcService,
+ final HighAvailabilityServices haServices,
+ final OnCompletionActions toNotifyOnComplete,
+ final FatalErrorHandler errorHandler) throws Exception
{
this(jobGraph, configuration, rpcService, haServices,
- JobManagerServices.fromConfiguration(configuration), toNotify);
+ new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)),
+ toNotifyOnComplete, errorHandler);
}
public JobManagerRunner(
- final JobGraph jobGraph,
- final Configuration configuration,
- final RpcService rpcService,
- final HighAvailabilityServices haServices,
- final JobManagerServices jobManagerServices,
- final OnCompletionActions toNotify) throws Exception
+ final JobGraph jobGraph,
+ final Configuration configuration,
+ final RpcService rpcService,
+ final HighAvailabilityServices haServices,
+ final MetricRegistry metricRegistry,
+ final OnCompletionActions toNotifyOnComplete,
+ final FatalErrorHandler errorHandler) throws Exception
+ {
+ this(jobGraph, configuration, rpcService, haServices,
+ JobManagerServices.fromConfiguration(configuration, haServices),
+ metricRegistry,
+ toNotifyOnComplete, errorHandler);
+ }
+
+ /**
+ *
+ * <p>Exceptions that occur while creating the JobManager or JobManagerRunner are directly
+ * thrown and not reported to the given {@code FatalErrorHandler}.
+ *
+ * <p>This JobManagerRunner assumes that it owns the given {@code JobManagerServices}.
+ * It will shut them down on error and on calls to {@link #shutdown()}.
+ *
+ * @throws Exception Thrown if the runner cannot be set up, because either one of the
+ * required services could not be started, ot the Job could not be initialized.
+ */
+ public JobManagerRunner(
+ final JobGraph jobGraph,
+ final Configuration configuration,
+ final RpcService rpcService,
+ final HighAvailabilityServices haServices,
+ final JobManagerServices jobManagerServices,
+ final MetricRegistry metricRegistry,
+ final OnCompletionActions toNotifyOnComplete,
+ final FatalErrorHandler errorHandler) throws Exception
{
- this.jobGraph = jobGraph;
- this.toNotify = toNotify;
- this.submittedJobGraphStore = haServices.getSubmittedJobGraphStore();
- this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());
-
- this.jobManager = new JobMaster(
- jobGraph, configuration, rpcService, haServices,
- jobManagerServices.libraryCacheManager,
- jobManagerServices.restartStrategyFactory,
- jobManagerServices.savepointStore,
- jobManagerServices.timeout,
- new Scheduler(jobManagerServices.executorService),
- jobManagerServices.jobManagerMetricGroup,
- this);
+
+ JobManagerMetricGroup jobManagerMetrics = null;
+
+ // make sure we cleanly shut down out JobManager services if initialization fails
+ try {
+ this.jobGraph = checkNotNull(jobGraph);
+ this.toNotifyOnComplete = checkNotNull(toNotifyOnComplete);
+ this.errorHandler = checkNotNull(errorHandler);
+ this.jobManagerServices = checkNotNull(jobManagerServices);
+
+ checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty");
+
+ final String hostAddress = rpcService.getAddress().isEmpty() ? "localhost" : rpcService.getAddress();
+ jobManagerMetrics = new JobManagerMetricGroup(metricRegistry, hostAddress);
+ this.jobManagerMetricGroup = jobManagerMetrics;
+
+ // libraries and class loader first
+ final BlobLibraryCacheManager libraryCacheManager = jobManagerServices.libraryCacheManager;
+ try {
+ libraryCacheManager.registerJob(
+ jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths());
+ } catch (IOException e) {
+ throw new Exception("Cannot set up the user code libraries: " + e.getMessage(), e);
+ }
+
+ final ClassLoader userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
+ if (userCodeLoader == null) {
+ throw new Exception("The user code class loader could not be initialized.");
+ }
+
+ // high availability services next
+ this.runningJobsRegistry = haServices.getRunningJobsRegistry();
+ this.leaderElectionService = haServices.getJobManagerLeaderElectionService(jobGraph.getJobID());
+
+ // now start the JobManager
+ this.jobManager = new JobMaster(
+ jobGraph, configuration,
+ rpcService,
+ haServices,
+ jobManagerServices.executorService,
+ jobManagerServices.libraryCacheManager,
+ jobManagerServices.restartStrategyFactory,
+ jobManagerServices.savepointStore,
+ jobManagerServices.rpcAskTimeout,
+ jobManagerMetrics,
+ this,
+ this,
+ userCodeLoader);
+ }
+ catch (Throwable t) {
+ // clean up everything
+ try {
+ jobManagerServices.shutdown();
+ } catch (Throwable tt) {
+ log.error("Error while shutting down JobManager services", tt);
+ }
+
+ if (jobManagerMetrics != null) {
+ jobManagerMetrics.close();
+ }
+
+ throw new JobExecutionException(jobGraph.getJobID(), "Could not set up JobManager", t);
+ }
}
//----------------------------------------------------------------------------------------------
@@ -101,9 +202,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
//----------------------------------------------------------------------------------------------
public void start() throws Exception {
- jobManager.init();
- jobManager.start();
-
try {
leaderElectionService.start(this);
}
@@ -114,11 +212,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
}
public void shutdown() {
- shutdown(new Exception("The JobManager runner is shutting down"));
- }
-
- public void shutdown(Throwable cause) {
- // TODO what is the cause used for ?
shutdownInternally();
}
@@ -129,12 +222,29 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
if (leaderElectionService != null) {
try {
leaderElectionService.stop();
- } catch (Exception e) {
- log.error("Could not properly shutdown the leader election service.");
+ } catch (Throwable t) {
+ log.error("Could not properly shutdown the leader election service", t);
}
}
- jobManager.shutDown();
+ try {
+ jobManager.shutDown();
+ } catch (Throwable t) {
+ log.error("Error shutting down JobManager", t);
+ }
+
+ try {
+ jobManagerServices.shutdown();
+ } catch (Throwable t) {
+ log.error("Error shutting down JobManager services", t);
+ }
+
+ // make all registered metrics go away
+ try {
+ jobManagerMetricGroup.close();
+ } catch (Throwable t) {
+ log.error("Error while unregistering metrics", t);
+ }
}
}
@@ -148,11 +258,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
@Override
public void jobFinished(JobExecutionResult result) {
try {
+ unregisterJobFromHighAvailability();
shutdownInternally();
}
finally {
- if (toNotify != null) {
- toNotify.jobFinished(result);
+ if (toNotifyOnComplete != null) {
+ toNotifyOnComplete.jobFinished(result);
}
}
}
@@ -163,11 +274,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
@Override
public void jobFailed(Throwable cause) {
try {
+ unregisterJobFromHighAvailability();
shutdownInternally();
}
finally {
- if (toNotify != null) {
- toNotify.jobFailed(cause);
+ if (toNotifyOnComplete != null) {
+ toNotifyOnComplete.jobFailed(cause);
}
}
}
@@ -178,11 +290,12 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
@Override
public void jobFinishedByOther() {
try {
+ unregisterJobFromHighAvailability();
shutdownInternally();
}
finally {
- if (toNotify != null) {
- toNotify.jobFinishedByOther();
+ if (toNotifyOnComplete != null) {
+ toNotifyOnComplete.jobFinishedByOther();
}
}
}
@@ -192,18 +305,43 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
*/
@Override
public void onFatalError(Throwable exception) {
- // first and in any case, notify our handler, so it can react fast
+ // we log first to make sure an explaining message goes into the log
+ // we even guard the log statement here to increase chances that the error handler
+ // gets the notification on hard critical situations like out-of-memory errors
+ try {
+ log.error("JobManager runner encountered a fatal error.", exception);
+ } catch (Throwable ignored) {}
+
+ // in any case, notify our handler, so it can react fast
try {
- if (toNotify != null) {
- toNotify.onFatalError(exception);
+ if (errorHandler != null) {
+ errorHandler.onFatalError(exception);
}
}
finally {
- log.error("JobManager runner encountered a fatal error.", exception);
+ // the shutdown may not even needed any more, if the fatal error
+ // handler kills the process. that is fine, a process kill cleans up better than anything.
shutdownInternally();
}
}
+ /**
+ * Marks this runner's job as not running. Other JobManager will not recover the job
+ * after this call.
+ *
+ * <p>This method never throws an exception.
+ */
+ private void unregisterJobFromHighAvailability() {
+ try {
+ runningJobsRegistry.setJobFinished(jobGraph.getJobID());
+ }
+ catch (Throwable t) {
+ log.error("Could not un-register from high-availability services job {} ({})." +
+ "Other JobManager's may attempt to recover it and re-execute it.",
+ jobGraph.getName(), jobGraph.getJobID(), t);
+ }
+ }
+
//----------------------------------------------------------------------------------------------
// Leadership methods
//----------------------------------------------------------------------------------------------
@@ -223,15 +361,25 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
// it's okay that job manager wait for the operation complete
leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+ boolean jobRunning;
+ try {
+ jobRunning = runningJobsRegistry.isJobRunning(jobGraph.getJobID());
+ } catch (Throwable t) {
+ log.error("Could not access status (running/finished) of job {}. " +
+ "Falling back to assumption that job is running and attempting recovery...",
+ jobGraph.getJobID(), t);
+ jobRunning = true;
+ }
+
// Double check the leadership after we confirm that, 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.
if (leaderElectionService.hasLeadership()) {
- if (isJobFinishedByOthers()) {
+ if (jobRunning) {
+ jobManager.start(leaderSessionID);
+ } else {
log.info("Job {} ({}) already finished by others.", jobGraph.getName(), jobGraph.getJobID());
jobFinishedByOther();
- } else {
- jobManager.getSelf().startJob(leaderSessionID);
}
}
}
@@ -248,7 +396,7 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
log.info("JobManager for job {} ({}) was revoked leadership at {}.",
jobGraph.getName(), jobGraph.getJobID(), getAddress());
- jobManager.getSelf().suspendJob(new Exception("JobManager is no longer the leader."));
+ jobManager.getSelf().suspendExecution(new Exception("JobManager is no longer the leader."));
}
}
@@ -263,11 +411,9 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions {
onFatalError(exception);
}
- @VisibleForTesting
- boolean isJobFinishedByOthers() {
- // TODO: Fix
- return false;
- }
+ //----------------------------------------------------------------------------------------------
+ // Testing
+ //----------------------------------------------------------------------------------------------
@VisibleForTesting
boolean isShutdown() {
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
index e6beba6..eebfbfa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java
@@ -19,13 +19,21 @@
package org.apache.flink.runtime.jobmaster;
import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.blob.BlobServer;
import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStoreFactory;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
-import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.util.ExceptionUtils;
+import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ForkJoinPool;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -42,32 +50,95 @@ public class JobManagerServices {
public final SavepointStore savepointStore;
- public final Time timeout;
-
- public final JobManagerMetricGroup jobManagerMetricGroup;
+ public final Time rpcAskTimeout;
public JobManagerServices(
ExecutorService executorService,
BlobLibraryCacheManager libraryCacheManager,
RestartStrategyFactory restartStrategyFactory,
SavepointStore savepointStore,
- Time timeout,
- JobManagerMetricGroup jobManagerMetricGroup) {
+ Time rpcAskTimeout) {
this.executorService = checkNotNull(executorService);
this.libraryCacheManager = checkNotNull(libraryCacheManager);
this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
this.savepointStore = checkNotNull(savepointStore);
- this.timeout = checkNotNull(timeout);
- this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
+ this.rpcAskTimeout = checkNotNull(rpcAskTimeout);
+ }
+
+ /**
+ *
+ * <p>This method makes sure all services are closed or shut down, even when an exception occurred
+ * in the shutdown of one component. The first encountered exception is thrown, with successive
+ * exceptions added as suppressed exceptions.
+ *
+ * @throws Exception The first Exception encountered during shutdown.
+ */
+ public void shutdown() throws Exception {
+ Throwable firstException = null;
+
+ try {
+ executorService.shutdownNow();
+ } catch (Throwable t) {
+ firstException = t;
+ }
+
+ try {
+ savepointStore.shutdown();
+ }
+ catch (Throwable t) {
+ if (firstException == null) {
+ firstException = t;
+ } else {
+ firstException.addSuppressed(t);
+ }
+ }
+
+ try {
+ libraryCacheManager.shutdown();
+ }
+ catch (Throwable t) {
+ if (firstException == null) {
+ firstException = t;
+ } else {
+ firstException.addSuppressed(t);
+ }
+ }
+
+ if (firstException != null) {
+ ExceptionUtils.rethrowException(firstException, "Error while shutting down JobManager services");
+ }
}
// ------------------------------------------------------------------------
// Creating the components from a configuration
// ------------------------------------------------------------------------
- public static JobManagerServices fromConfiguration(Configuration config) throws Exception {
- // TODO not yet implemented
- return null;
+
+ public static JobManagerServices fromConfiguration(
+ Configuration config,
+ HighAvailabilityServices haServices) throws Exception {
+
+ final BlobServer blobServer = new BlobServer(config, haServices);
+
+ final long cleanupInterval = config.getLong(
+ ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+ ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+ final BlobLibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(blobServer, cleanupInterval);
+
+ final FiniteDuration timeout;
+ try {
+ timeout = AkkaUtils.getTimeout(config);
+ } catch (NumberFormatException e) {
+ throw new IllegalConfigurationException(AkkaUtils.formatDurationParingErrorMessage());
+ }
+
+ return new JobManagerServices(
+ new ForkJoinPool(),
+ libraryCacheManager,
+ RestartStrategyFactory.createRestartStrategyFactory(config),
+ SavepointStoreFactory.createFromConfig(config),
+ Time.of(timeout.length(), timeout.unit()));
}
}
[7/8] flink git commit: [FLINK-4764] [core] Introduce Config Options
Posted by se...@apache.org.
[FLINK-4764] [core] Introduce Config Options
This is a more concise and maintainable way to define configuration keys, default values,
deprecated keys, etc.
This closes #2605
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d71a09cc
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d71a09cc
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d71a09cc
Branch: refs/heads/flip-6
Commit: d71a09cc2a36a877e8287db8d9fe84134a4901ba
Parents: 05436f4
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 7 15:24:44 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200
----------------------------------------------------------------------
.../flink/configuration/ConfigOption.java | 171 ++++++++
.../flink/configuration/ConfigOptions.java | 116 ++++++
.../flink/configuration/Configuration.java | 407 +++++++++++++++----
.../configuration/DelegatingConfiguration.java | 118 +++++-
.../flink/configuration/ConfigurationTest.java | 95 ++++-
.../DelegatingConfigurationTest.java | 55 +--
.../UnmodifiableConfigurationTest.java | 16 +-
7 files changed, 844 insertions(+), 134 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
new file mode 100644
index 0000000..3531f6d
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOption.java
@@ -0,0 +1,171 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A {@code ConfigOption} describes a configuration parameter. It encapsulates
+ * the configuration key, deprecated older versions of the key, and an optional
+ * default value for the configuration parameter.
+ *
+ * <p>{@code ConfigOptions} are built via the {@link ConfigOptions} class.
+ * Once created, a config option is immutable.
+ *
+ * @param <T> The type of value associated with the configuration option.
+ */
+@PublicEvolving
+public class ConfigOption<T> {
+
+ private static final String[] EMPTY = new String[0];
+
+ // ------------------------------------------------------------------------
+
+ /** The current key for that config option */
+ private final String key;
+
+ /** The list of deprecated keys, in the order to be checked */
+ private final String[] deprecatedKeys;
+
+ /** The default value for this option */
+ private final T defaultValue;
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a new config option with no deprecated keys.
+ *
+ * @param key The current key for that config option
+ * @param defaultValue The default value for this option
+ */
+ ConfigOption(String key, T defaultValue) {
+ this.key = checkNotNull(key);
+ this.defaultValue = defaultValue;
+ this.deprecatedKeys = EMPTY;
+ }
+
+ /**
+ * Creates a new config option with deprecated keys.
+ *
+ * @param key The current key for that config option
+ * @param defaultValue The default value for this option
+ * @param deprecatedKeys The list of deprecated keys, in the order to be checked
+ */
+ ConfigOption(String key, T defaultValue, String... deprecatedKeys) {
+ this.key = checkNotNull(key);
+ this.defaultValue = defaultValue;
+ this.deprecatedKeys = deprecatedKeys == null || deprecatedKeys.length == 0 ? EMPTY : deprecatedKeys;
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a new config option, using this option's key and default value, and
+ * adding the given deprecated keys.
+ *
+ * <p>When obtaining a value from the configuration via {@link Configuration#getValue(ConfigOption)},
+ * the deprecated keys will be checked in the order provided to this method. The first key for which
+ * a value is found will be used - that value will be returned.
+ *
+ * @param deprecatedKeys The deprecated keys, in the order in which they should be checked.
+ * @return A new config options, with the given deprecated keys.
+ */
+ public ConfigOption<T> withDeprecatedKeys(String... deprecatedKeys) {
+ return new ConfigOption<>(key, defaultValue, deprecatedKeys);
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the configuration key.
+ * @return The configuration key
+ */
+ public String key() {
+ return key;
+ }
+
+ /**
+ * Checks if this option has a default value.
+ * @return True if it has a default value, false if not.
+ */
+ public boolean hasDefaultValue() {
+ return defaultValue != null;
+ }
+
+ /**
+ * Returns the default value, or null, if there is no default value.
+ * @return The default value, or null.
+ */
+ public T defaultValue() {
+ return defaultValue;
+ }
+
+ /**
+ * Checks whether this option has deprecated keys.
+ * @return True if the option has deprecated keys, false if not.
+ */
+ public boolean hasDeprecatedKeys() {
+ return deprecatedKeys != EMPTY;
+ }
+
+ /**
+ * Gets the deprecated keys, in the order to be checked.
+ * @return The option's deprecated keys.
+ */
+ public Iterable<String> deprecatedKeys() {
+ return deprecatedKeys == EMPTY ? Collections.<String>emptyList() : Arrays.asList(deprecatedKeys);
+ }
+
+ // ------------------------------------------------------------------------
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ else if (o != null && o.getClass() == ConfigOption.class) {
+ ConfigOption<?> that = (ConfigOption<?>) o;
+ return this.key.equals(that.key) &&
+ Arrays.equals(this.deprecatedKeys, that.deprecatedKeys) &&
+ (this.defaultValue == null ? that.defaultValue == null :
+ (that.defaultValue != null && this.defaultValue.equals(that.defaultValue)));
+ }
+ else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * key.hashCode() +
+ 17 * Arrays.hashCode(deprecatedKeys) +
+ (defaultValue != null ? defaultValue.hashCode() : 0);
+ }
+
+ @Override
+ public String toString() {
+ return String.format("Key: '%s' , default: %s (deprecated keys: %s)",
+ key, defaultValue, Arrays.toString(deprecatedKeys));
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
new file mode 100644
index 0000000..f87da0a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigOptions.java
@@ -0,0 +1,116 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@code ConfigOptions} are used to build a {@link ConfigOption}.
+ * The option is typically built in one of the following pattern:
+ *
+ * <pre>{@code
+ * // simple string-valued option with a default value
+ * ConfigOption<String> tempDirs = ConfigOptions
+ * .key("tmp.dir")
+ * .defaultValue("/tmp");
+ *
+ * // simple integer-valued option with a default value
+ * ConfigOption<Integer> parallelism = ConfigOptions
+ * .key("application.parallelism")
+ * .defaultValue(100);
+ *
+ * // option with no default value
+ * ConfigOption<String> userName = ConfigOptions
+ * .key("user.name")
+ * .noDefaultValue();
+ *
+ * // option with deprecated keys to check
+ * ConfigOption<Double> threshold = ConfigOptions
+ * .key("cpu.utilization.threshold")
+ * .defaultValue(0.9).
+ * .withDeprecatedKeys("cpu.threshold");
+ * }</pre>
+ */
+@PublicEvolving
+public class ConfigOptions {
+
+ /**
+ * Starts building a new {@link ConfigOption}.
+ *
+ * @param key The key for the config option.
+ * @return The builder for the config option with the given key.
+ */
+ public static OptionBuilder key(String key) {
+ checkNotNull(key);
+ return new OptionBuilder(key);
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * The option builder is used to create a {@link ConfigOption}.
+ * It is instantiated via {@link ConfigOptions#key(String)}.
+ */
+ public static final class OptionBuilder {
+
+ /** The key for the config option */
+ private final String key;
+
+ /**
+ * Creates a new OptionBuilder.
+ * @param key The key for the config option
+ */
+ OptionBuilder(String key) {
+ this.key = key;
+ }
+
+ /**
+ * Creates a ConfigOption with the given default value.
+ *
+ * <p>This method does not accept "null". For options with no default value, choose
+ * one of the {@code noDefaultValue} methods.
+ *
+ * @param value The default value for the config option
+ * @param <T> The type of the default value.
+ * @return The config option with the default value.
+ */
+ public <T> ConfigOption<T> defaultValue(T value) {
+ checkNotNull(value);
+ return new ConfigOption<T>(key, value);
+ }
+
+ /**
+ * Creates a string-valued option with no default value.
+ * String-valued options are the only ones that can have no
+ * default value.
+ *
+ * @return The created ConfigOption.
+ */
+ public ConfigOption<String> noDefaultValue() {
+ return new ConfigOption<>(key, null);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ /** Not intended to be instantiated */
+ private ConfigOptions() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
index 8ca5d07..f15c669 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java
@@ -27,6 +27,7 @@ import java.util.Properties;
import java.util.Set;
import org.apache.flink.annotation.Public;
+import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
@@ -134,7 +135,33 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
return o.toString();
}
}
-
+
+ /**
+ * Returns the value associated with the given config option as a string.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public String getString(ConfigOption<String> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return o == null ? null : o.toString();
+ }
+
+ /**
+ * Returns the value associated with the given config option as a string.
+ * If no value is mapped under any key of the option, it returns the specified
+ * default instead of the option's default value.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public String getString(ConfigOption<String> configOption, String overrideDefault) {
+ Object o = getRawValueFromOption(configOption);
+ return o == null ? overrideDefault : o.toString();
+ }
+
/**
* Adds the given key/value pair to the configuration object.
*
@@ -148,6 +175,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
}
/**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setString(ConfigOption<String> key, String value) {
+ setValueInternal(key.key(), value);
+ }
+
+ /**
* Returns the value associated with the given key as an integer.
*
* @param key
@@ -161,28 +202,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
if (o == null) {
return defaultValue;
}
-
- if (o.getClass() == Integer.class) {
- return (Integer) o;
- }
- else if (o.getClass() == Long.class) {
- long value = (Long) o;
- if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
- return (int) value;
- } else {
- LOG.warn("Configuration value {} overflows/underflows the integer type.", value);
- return defaultValue;
- }
- }
- else {
- try {
- return Integer.parseInt(o.toString());
- }
- catch (NumberFormatException e) {
- LOG.warn("Configuration cannot evaluate value {} as an integer number", o);
- return defaultValue;
- }
- }
+
+ return convertToInt(o, defaultValue);
+ }
+
+ /**
+ * Returns the value associated with the given config option as an integer.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public int getInteger(ConfigOption<Integer> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return convertToInt(o, configOption.defaultValue());
}
/**
@@ -198,6 +231,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
}
/**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setInteger(ConfigOption<Integer> key, int value) {
+ setValueInternal(key.key(), value);
+ }
+
+ /**
* Returns the value associated with the given key as a long.
*
* @param key
@@ -211,22 +258,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
if (o == null) {
return defaultValue;
}
-
- if (o.getClass() == Long.class) {
- return (Long) o;
- }
- else if (o.getClass() == Integer.class) {
- return ((Integer) o).longValue();
- }
- else {
- try {
- return Long.parseLong(o.toString());
- }
- catch (NumberFormatException e) {
- LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number");
- return defaultValue;
- }
- }
+
+ return convertToLong(o, defaultValue);
+ }
+
+ /**
+ * Returns the value associated with the given config option as a long integer.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public long getLong(ConfigOption<Long> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return convertToLong(o, configOption.defaultValue());
}
/**
@@ -242,6 +287,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
}
/**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setLong(ConfigOption<Long> key, long value) {
+ setValueInternal(key.key(), value);
+ }
+
+ /**
* Returns the value associated with the given key as a boolean.
*
* @param key
@@ -255,13 +314,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
if (o == null) {
return defaultValue;
}
-
- if (o.getClass() == Boolean.class) {
- return (Boolean) o;
- }
- else {
- return Boolean.parseBoolean(o.toString());
- }
+
+ return convertToBoolean(o);
+ }
+
+ /**
+ * Returns the value associated with the given config option as a boolean.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public boolean getBoolean(ConfigOption<Boolean> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return convertToBoolean(o);
}
/**
@@ -277,6 +343,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
}
/**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setBoolean(ConfigOption<Boolean> key, boolean value) {
+ setValueInternal(key.key(), value);
+ }
+
+ /**
* Returns the value associated with the given key as a float.
*
* @param key
@@ -290,28 +370,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
if (o == null) {
return defaultValue;
}
-
- if (o.getClass() == Float.class) {
- return (Float) o;
- }
- else if (o.getClass() == Double.class) {
- double value = ((Double) o);
- if (value <= Float.MAX_VALUE && value >= Float.MIN_VALUE) {
- return (float) value;
- } else {
- LOG.warn("Configuration value {} overflows/underflows the float type.", value);
- return defaultValue;
- }
- }
- else {
- try {
- return Float.parseFloat(o.toString());
- }
- catch (NumberFormatException e) {
- LOG.warn("Configuration cannot evaluate value {} as a float value", o);
- return defaultValue;
- }
- }
+
+ return convertToFloat(o, defaultValue);
+ }
+
+ /**
+ * Returns the value associated with the given config option as a float.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public float getFloat(ConfigOption<Float> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return convertToFloat(o, configOption.defaultValue());
}
/**
@@ -325,7 +397,21 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
public void setFloat(String key, float value) {
setValueInternal(key, value);
}
-
+
+ /**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setFloat(ConfigOption<Float> key, float value) {
+ setValueInternal(key.key(), value);
+ }
+
/**
* Returns the value associated with the given key as a double.
*
@@ -340,22 +426,20 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
if (o == null) {
return defaultValue;
}
-
- if (o.getClass() == Double.class) {
- return (Double) o;
- }
- else if (o.getClass() == Float.class) {
- return ((Float) o).doubleValue();
- }
- else {
- try {
- return Double.parseDouble(o.toString());
- }
- catch (NumberFormatException e) {
- LOG.warn("Configuration cannot evaluate value {} as a double value", o);
- return defaultValue;
- }
- }
+
+ return convertToDouble(o, defaultValue);
+ }
+
+ /**
+ * Returns the value associated with the given config option as a {@code double}.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public double getDouble(ConfigOption<Double> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return convertToDouble(o, configOption.defaultValue());
}
/**
@@ -369,7 +453,21 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
public void setDouble(String key, double value) {
setValueInternal(key, value);
}
-
+
+ /**
+ * Adds the given value to the configuration object.
+ * The main key of the config option will be used to map the value.
+ *
+ * @param key
+ * the option specifying the key to be added
+ * @param value
+ * the value of the key/value pair to be added
+ */
+ @PublicEvolving
+ public void setDouble(ConfigOption<Double> key, double value) {
+ setValueInternal(key.key(), value);
+ }
+
/**
* Returns the value associated with the given key as a byte array.
*
@@ -407,6 +505,18 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
setValueInternal(key, bytes);
}
+ /**
+ * Returns the value associated with the given config option as a string.
+ *
+ * @param configOption The configuration option
+ * @return the (default) value associated with the given config option
+ */
+ @PublicEvolving
+ public String getValue(ConfigOption<?> configOption) {
+ Object o = getValueOrDefaultFromOption(configOption);
+ return o == null ? null : o.toString();
+ }
+
// --------------------------------------------------------------------------------------------
/**
@@ -523,7 +633,130 @@ public class Configuration extends ExecutionConfig.GlobalJobParameters
return this.confData.get(key);
}
}
-
+
+ private Object getRawValueFromOption(ConfigOption<?> configOption) {
+ // first try the current key
+ Object o = getRawValue(configOption.key());
+
+ if (o != null) {
+ return o;
+ }
+ else if (configOption.hasDeprecatedKeys()) {
+ for (String deprecatedKey : configOption.deprecatedKeys()) {
+ Object oo = getRawValue(deprecatedKey);
+ if (oo != null) {
+ return oo;
+ }
+ }
+ }
+
+ return null;
+ }
+
+ private Object getValueOrDefaultFromOption(ConfigOption<?> configOption) {
+ Object o = getRawValueFromOption(configOption);
+ return o != null ? o : configOption.defaultValue();
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Type conversion
+ // --------------------------------------------------------------------------------------------
+
+ private int convertToInt(Object o, int defaultValue) {
+ if (o.getClass() == Integer.class) {
+ return (Integer) o;
+ }
+ else if (o.getClass() == Long.class) {
+ long value = (Long) o;
+ if (value <= Integer.MAX_VALUE && value >= Integer.MIN_VALUE) {
+ return (int) value;
+ } else {
+ LOG.warn("Configuration value {} overflows/underflows the integer type.", value);
+ return defaultValue;
+ }
+ }
+ else {
+ try {
+ return Integer.parseInt(o.toString());
+ }
+ catch (NumberFormatException e) {
+ LOG.warn("Configuration cannot evaluate value {} as an integer number", o);
+ return defaultValue;
+ }
+ }
+ }
+
+ private long convertToLong(Object o, long defaultValue) {
+ if (o.getClass() == Long.class) {
+ return (Long) o;
+ }
+ else if (o.getClass() == Integer.class) {
+ return ((Integer) o).longValue();
+ }
+ else {
+ try {
+ return Long.parseLong(o.toString());
+ }
+ catch (NumberFormatException e) {
+ LOG.warn("Configuration cannot evaluate value " + o + " as a long integer number");
+ return defaultValue;
+ }
+ }
+ }
+
+ private boolean convertToBoolean(Object o) {
+ if (o.getClass() == Boolean.class) {
+ return (Boolean) o;
+ }
+ else {
+ return Boolean.parseBoolean(o.toString());
+ }
+ }
+
+ private float convertToFloat(Object o, float defaultValue) {
+ if (o.getClass() == Float.class) {
+ return (Float) o;
+ }
+ else if (o.getClass() == Double.class) {
+ double value = ((Double) o);
+ if (value <= Float.MAX_VALUE && value >= Float.MIN_VALUE) {
+ return (float) value;
+ } else {
+ LOG.warn("Configuration value {} overflows/underflows the float type.", value);
+ return defaultValue;
+ }
+ }
+ else {
+ try {
+ return Float.parseFloat(o.toString());
+ }
+ catch (NumberFormatException e) {
+ LOG.warn("Configuration cannot evaluate value {} as a float value", o);
+ return defaultValue;
+ }
+ }
+ }
+
+ private double convertToDouble(Object o, double defaultValue) {
+ if (o.getClass() == Double.class) {
+ return (Double) o;
+ }
+ else if (o.getClass() == Float.class) {
+ return ((Float) o).doubleValue();
+ }
+ else {
+ try {
+ return Double.parseDouble(o.toString());
+ }
+ catch (NumberFormatException e) {
+ LOG.warn("Configuration cannot evaluate value {} as a double value", o);
+ return defaultValue;
+ }
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Serialization
// --------------------------------------------------------------------------------------------
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
index dba77f3..bd9a962 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/DelegatingConfiguration.java
@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.flink.configuration;
import org.apache.flink.core.memory.DataInputView;
@@ -22,7 +23,11 @@ import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.util.Preconditions;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
+import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
@@ -55,8 +60,7 @@ public final class DelegatingConfiguration extends Configuration {
* @param backingConfig The configuration holding the actual config data.
* @param prefix The prefix prepended to all config keys.
*/
- public DelegatingConfiguration(Configuration backingConfig, String prefix)
- {
+ public DelegatingConfiguration(Configuration backingConfig, String prefix) {
this.backingConfig = Preconditions.checkNotNull(backingConfig);
this.prefix = prefix;
}
@@ -69,11 +73,26 @@ public final class DelegatingConfiguration extends Configuration {
}
@Override
+ public String getString(ConfigOption<String> configOption) {
+ return this.backingConfig.getString(prefixOption(configOption, prefix));
+ }
+
+ @Override
+ public String getString(ConfigOption<String> configOption, String overrideDefault) {
+ return this.backingConfig.getString(prefixOption(configOption, prefix), overrideDefault);
+ }
+
+ @Override
public void setString(String key, String value) {
this.backingConfig.setString(this.prefix + key, value);
}
@Override
+ public void setString(ConfigOption<String> key, String value) {
+ this.backingConfig.setString(prefix + key.key(), value);
+ }
+
+ @Override
public <T> Class<T> getClass(String key, Class<? extends T> defaultValue, ClassLoader classLoader) throws ClassNotFoundException {
return this.backingConfig.getClass(this.prefix + key, defaultValue, classLoader);
}
@@ -89,51 +108,101 @@ public final class DelegatingConfiguration extends Configuration {
}
@Override
+ public int getInteger(ConfigOption<Integer> configOption) {
+ return this.backingConfig.getInteger(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void setInteger(String key, int value) {
this.backingConfig.setInteger(this.prefix + key, value);
}
@Override
+ public void setInteger(ConfigOption<Integer> key, int value) {
+ this.backingConfig.setInteger(prefix + key.key(), value);
+ }
+
+ @Override
public long getLong(String key, long defaultValue) {
return this.backingConfig.getLong(this.prefix + key, defaultValue);
}
@Override
+ public long getLong(ConfigOption<Long> configOption) {
+ return this.backingConfig.getLong(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void setLong(String key, long value) {
this.backingConfig.setLong(this.prefix + key, value);
}
@Override
+ public void setLong(ConfigOption<Long> key, long value) {
+ this.backingConfig.setLong(prefix + key.key(), value);
+ }
+
+ @Override
public boolean getBoolean(String key, boolean defaultValue) {
return this.backingConfig.getBoolean(this.prefix + key, defaultValue);
}
@Override
+ public boolean getBoolean(ConfigOption<Boolean> configOption) {
+ return this.backingConfig.getBoolean(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void setBoolean(String key, boolean value) {
this.backingConfig.setBoolean(this.prefix + key, value);
}
@Override
+ public void setBoolean(ConfigOption<Boolean> key, boolean value) {
+ this.backingConfig.setBoolean(prefix + key.key(), value);
+ }
+
+ @Override
public float getFloat(String key, float defaultValue) {
return this.backingConfig.getFloat(this.prefix + key, defaultValue);
}
@Override
+ public float getFloat(ConfigOption<Float> configOption) {
+ return this.backingConfig.getFloat(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void setFloat(String key, float value) {
this.backingConfig.setFloat(this.prefix + key, value);
}
@Override
+ public void setFloat(ConfigOption<Float> key, float value) {
+ this.backingConfig.setFloat(prefix + key.key(), value);
+ }
+
+ @Override
public double getDouble(String key, double defaultValue) {
return this.backingConfig.getDouble(this.prefix + key, defaultValue);
}
@Override
+ public double getDouble(ConfigOption<Double> configOption) {
+ return this.backingConfig.getDouble(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void setDouble(String key, double value) {
this.backingConfig.setDouble(this.prefix + key, value);
}
@Override
+ public void setDouble(ConfigOption<Double> key, double value) {
+ this.backingConfig.setDouble(prefix + key.key(), value);
+ }
+
+ @Override
public byte[] getBytes(final String key, final byte[] defaultValue) {
return this.backingConfig.getBytes(this.prefix + key, defaultValue);
}
@@ -144,6 +213,11 @@ public final class DelegatingConfiguration extends Configuration {
}
@Override
+ public String getValue(ConfigOption<?> configOption) {
+ return this.backingConfig.getValue(prefixOption(configOption, prefix));
+ }
+
+ @Override
public void addAllToProperties(Properties props) {
// only add keys with our prefix
synchronized (backingConfig.confData) {
@@ -195,6 +269,27 @@ public final class DelegatingConfiguration extends Configuration {
return set;
}
+ @Override
+ public Configuration clone() {
+ return new DelegatingConfiguration(backingConfig.clone(), prefix);
+ }
+
+ @Override
+ public Map<String, String> toMap() {
+ Map<String, String> map = backingConfig.toMap();
+ Map<String, String> prefixed = new HashMap<>(map.size());
+ for (Map.Entry<String, String> entry : map.entrySet()) {
+ prefixed.put(prefix + entry.getKey(), entry.getValue());
+ }
+
+ return prefixed;
+ }
+
+ @Override
+ public boolean containsKey(String key) {
+ return backingConfig.containsKey(prefix + key);
+ }
+
// --------------------------------------------------------------------------------------------
@Override
@@ -225,4 +320,23 @@ public final class DelegatingConfiguration extends Configuration {
return false;
}
}
+
+ // --------------------------------------------------------------------------------------------
+
+ private static <T> ConfigOption<T> prefixOption(ConfigOption<T> option, String prefix) {
+ String key = prefix + option.key();
+
+ List<String> deprecatedKeys;
+ if (option.hasDeprecatedKeys()) {
+ deprecatedKeys = new ArrayList<>();
+ for (String dk : option.deprecatedKeys()) {
+ deprecatedKeys.add(prefix + dk);
+ }
+ } else {
+ deprecatedKeys = Collections.emptyList();
+ }
+
+ String[] deprecated = deprecatedKeys.toArray(new String[deprecatedKeys.size()]);
+ return new ConfigOption<T>(key, option.defaultValue(), deprecated);
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
index cf3c908..91c5f65 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/ConfigurationTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.configuration;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -33,7 +34,7 @@ import org.junit.Test;
* objects is tested.
*/
public class ConfigurationTest extends TestLogger {
-
+
private static final byte[] EMPTY_BYTES = new byte[0];
private static final long TOO_LONG = Integer.MAX_VALUE + 10L;
private static final double TOO_LONG_DOUBLE = Double.MAX_VALUE;
@@ -73,7 +74,7 @@ public class ConfigurationTest extends TestLogger {
fail(e.getMessage());
}
}
-
+
@Test
public void testConversions() {
try {
@@ -175,7 +176,7 @@ public class ConfigurationTest extends TestLogger {
fail(e.getMessage());
}
}
-
+
@Test
public void testCopyConstructor() {
try {
@@ -194,4 +195,92 @@ public class ConfigurationTest extends TestLogger {
fail(e.getMessage());
}
}
+
+ @Test
+ public void testOptionWithDefault() {
+ Configuration cfg = new Configuration();
+ cfg.setInteger("int-key", 11);
+ cfg.setString("string-key", "abc");
+
+ ConfigOption<String> presentStringOption = ConfigOptions.key("string-key").defaultValue("my-beautiful-default");
+ ConfigOption<Integer> presentIntOption = ConfigOptions.key("int-key").defaultValue(87);
+
+ assertEquals("abc", cfg.getString(presentStringOption));
+ assertEquals("abc", cfg.getValue(presentStringOption));
+
+ assertEquals(11, cfg.getInteger(presentIntOption));
+ assertEquals("11", cfg.getValue(presentIntOption));
+
+ // test getting default when no value is present
+
+ ConfigOption<String> stringOption = ConfigOptions.key("test").defaultValue("my-beautiful-default");
+ ConfigOption<Integer> intOption = ConfigOptions.key("test2").defaultValue(87);
+
+ // getting strings with default value should work
+ assertEquals("my-beautiful-default", cfg.getValue(stringOption));
+ assertEquals("my-beautiful-default", cfg.getString(stringOption));
+
+ // overriding the default should work
+ assertEquals("override", cfg.getString(stringOption, "override"));
+
+ // getting a primitive with a default value should work
+ assertEquals(87, cfg.getInteger(intOption));
+ assertEquals("87", cfg.getValue(intOption));
+ }
+
+ @Test
+ public void testOptionWithNoDefault() {
+ Configuration cfg = new Configuration();
+ cfg.setInteger("int-key", 11);
+ cfg.setString("string-key", "abc");
+
+ ConfigOption<String> presentStringOption = ConfigOptions.key("string-key").noDefaultValue();
+
+ assertEquals("abc", cfg.getString(presentStringOption));
+ assertEquals("abc", cfg.getValue(presentStringOption));
+
+ // test getting default when no value is present
+
+ ConfigOption<String> stringOption = ConfigOptions.key("test").noDefaultValue();
+
+ // getting strings for null should work
+ assertNull(cfg.getValue(stringOption));
+ assertNull(cfg.getString(stringOption));
+
+ // overriding the null default should work
+ assertEquals("override", cfg.getString(stringOption, "override"));
+ }
+
+ @Test
+ public void testDeprecatedKeys() {
+ Configuration cfg = new Configuration();
+ cfg.setInteger("the-key", 11);
+ cfg.setInteger("old-key", 12);
+ cfg.setInteger("older-key", 13);
+
+ ConfigOption<Integer> matchesFirst = ConfigOptions
+ .key("the-key")
+ .defaultValue(-1)
+ .withDeprecatedKeys("old-key", "older-key");
+
+ ConfigOption<Integer> matchesSecond = ConfigOptions
+ .key("does-not-exist")
+ .defaultValue(-1)
+ .withDeprecatedKeys("old-key", "older-key");
+
+ ConfigOption<Integer> matchesThird = ConfigOptions
+ .key("does-not-exist")
+ .defaultValue(-1)
+ .withDeprecatedKeys("foo", "older-key");
+
+ ConfigOption<Integer> notContained = ConfigOptions
+ .key("does-not-exist")
+ .defaultValue(-1)
+ .withDeprecatedKeys("not-there", "also-not-there");
+
+ assertEquals(11, cfg.getInteger(matchesFirst));
+ assertEquals(12, cfg.getInteger(matchesSecond));
+ assertEquals(13, cfg.getInteger(matchesThird));
+ assertEquals(-1, cfg.getInteger(notContained));
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
index d8b782d..9298a14 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/DelegatingConfigurationTest.java
@@ -24,8 +24,6 @@ import org.junit.Test;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Modifier;
-import java.util.Arrays;
-import java.util.Comparator;
import java.util.Set;
import static org.junit.Assert.assertTrue;
@@ -34,60 +32,43 @@ import static org.junit.Assert.assertEquals;
public class DelegatingConfigurationTest {
- /**
- * http://stackoverflow.com/questions/22225663/checking-in-a-unit-test-whether-all-methods-are-delegated
- */
@Test
public void testIfDelegatesImplementAllMethods() throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
- Comparator<Method> methodComparator = new Comparator<Method>() {
- @Override
- public int compare(Method o1, Method o2) {
- String o1Str = o1.getName() + typeParamToString(o1.getParameterTypes());
- String o2Str = o2.getName() + typeParamToString(o2.getParameterTypes());
- return o1Str.compareTo( o2Str );
- }
-
- private String typeParamToString(Class<?>[] classes) {
- String str = "";
- for(Object t : classes) {
- str += t.toString();
- }
- return str;
- }
- };
-
// For each method in the Configuration class...
Method[] confMethods = Configuration.class.getDeclaredMethods();
Method[] delegateMethods = DelegatingConfiguration.class.getDeclaredMethods();
- Arrays.sort(confMethods, methodComparator);
- Arrays.sort(delegateMethods, methodComparator);
- match : for (Method configurationMethod : confMethods) {
- boolean hasMethod = false;
- if(!Modifier.isPublic(configurationMethod.getModifiers()) ) {
+
+ for (Method configurationMethod : confMethods) {
+ if (!Modifier.isPublic(configurationMethod.getModifiers()) ) {
continue;
}
+
+ boolean hasMethod = false;
+
// Find matching method in wrapper class and call it
- mismatch: for (Method wrapperMethod : delegateMethods) {
+ lookForWrapper: for (Method wrapperMethod : delegateMethods) {
if (configurationMethod.getName().equals(wrapperMethod.getName())) {
-
+
// Get parameters for method
Class<?>[] wrapperMethodParams = wrapperMethod.getParameterTypes();
Class<?>[] configMethodParams = configurationMethod.getParameterTypes();
- if(wrapperMethodParams.length != configMethodParams.length) {
- System.err.println("Length");
- break mismatch;
+ if (wrapperMethodParams.length != configMethodParams.length) {
+ continue;
}
- for(int i = 0; i < wrapperMethodParams.length; i++) {
- if(wrapperMethodParams[i] != configMethodParams[i]) {
- break mismatch;
+
+ for (int i = 0; i < wrapperMethodParams.length; i++) {
+ if (wrapperMethodParams[i] != configMethodParams[i]) {
+ continue lookForWrapper;
}
}
hasMethod = true;
- break match;
+ break;
}
}
- assertTrue("Foo method '" + configurationMethod.getName() + "' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
+
+ assertTrue("Configuration method '" + configurationMethod.getName() +
+ "' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d71a09cc/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
index 386d03b..26e3d7a 100644
--- a/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
+++ b/flink-core/src/test/java/org/apache/flink/configuration/UnmodifiableConfigurationTest.java
@@ -56,6 +56,9 @@ public class UnmodifiableConfigurationTest extends TestLogger {
@Test
public void testExceptionOnSet() {
try {
+ @SuppressWarnings("rawtypes")
+ final ConfigOption rawOption = ConfigOptions.key("testkey").defaultValue("value");
+
Map<Class<?>, Object> parameters = new HashMap<Class<?>, Object>();
parameters.put(byte[].class, new byte[0]);
parameters.put(Class.class, Object.class);
@@ -65,19 +68,22 @@ public class UnmodifiableConfigurationTest extends TestLogger {
parameters.put(double.class, 0.0);
parameters.put(String.class, "");
parameters.put(boolean.class, false);
-
+
Class<UnmodifiableConfiguration> clazz = UnmodifiableConfiguration.class;
UnmodifiableConfiguration config = new UnmodifiableConfiguration(new Configuration());
-
+
for (Method m : clazz.getMethods()) {
if (m.getName().startsWith("set")) {
-
+
+ Class<?> keyClass = m.getParameterTypes()[0];
Class<?> parameterClass = m.getParameterTypes()[1];
+ Object key = keyClass == String.class ? "key" : rawOption;
+
Object parameter = parameters.get(parameterClass);
assertNotNull("method " + m + " not covered by test", parameter);
-
+
try {
- m.invoke(config, "key", parameter);
+ m.invoke(config, key, parameter);
fail("should fail with an exception");
}
catch (InvocationTargetException e) {
[6/8] flink git commit: [FLINK-4776] [distributed coordination] Move
ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder
Posted by se...@apache.org.
[FLINK-4776] [distributed coordination] Move ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/05436f4b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/05436f4b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/05436f4b
Branch: refs/heads/flip-6
Commit: 05436f4b64e771b22f13f56ff9e0ea7aa94b4ff7
Parents: 21b9f16
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 7 19:58:24 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200
----------------------------------------------------------------------
.../checkpoint/CheckpointCoordinator.java | 4 +-
.../runtime/executiongraph/ExecutionGraph.java | 8 +-
.../executiongraph/ExecutionGraphBuilder.java | 262 +++++++++++++++++++
.../flink/runtime/jobmanager/JobManager.scala | 168 ++----------
4 files changed, 297 insertions(+), 145 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
----------------------------------------------------------------------
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 4428427..e95afe0 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
@@ -171,7 +171,7 @@ public class CheckpointCoordinator {
CheckpointIDCounter checkpointIDCounter,
CompletedCheckpointStore completedCheckpointStore,
SavepointStore savepointStore,
- CheckpointStatsTracker statsTracker) throws Exception {
+ CheckpointStatsTracker statsTracker) {
// sanity checks
checkArgument(baseInterval > 0, "Checkpoint timeout must be larger than zero");
@@ -207,7 +207,7 @@ public class CheckpointCoordinator {
// issues a blocking call to ZooKeeper.
checkpointIDCounter.start();
} catch (Throwable t) {
- throw new Exception("Failed to start checkpoint ID counter: " + t.getMessage(), t);
+ throw new RuntimeException("Failed to start checkpoint ID counter: " + t.getMessage(), t);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
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 6023205..cf98ca6 100644
--- 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
@@ -359,7 +359,7 @@ public class ExecutionGraph {
CheckpointIDCounter checkpointIDCounter,
CompletedCheckpointStore checkpointStore,
SavepointStore savepointStore,
- CheckpointStatsTracker statsTracker) throws Exception {
+ CheckpointStatsTracker statsTracker) {
// simple sanity checks
if (interval < 10 || checkpointTimeout < 10) {
@@ -374,7 +374,11 @@ public class ExecutionGraph {
ExecutionVertex[] tasksToCommitTo = collectExecutionVertices(verticesToCommitTo);
// disable to make sure existing checkpoint coordinators are cleared
- disableSnaphotCheckpointing();
+ try {
+ disableSnaphotCheckpointing();
+ } catch (Throwable t) {
+ LOG.error("Error while shutting down checkpointer.");
+ }
checkpointStatsTracker = Objects.requireNonNull(statsTracker, "Checkpoint stats tracker");
http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
new file mode 100644
index 0000000..1c6eb8d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphBuilder.java
@@ -0,0 +1,262 @@
+/*
+ * 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.executiongraph;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
+import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
+import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
+import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.client.JobSubmissionException;
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
+import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.duration.FiniteDuration;
+
+import javax.annotation.Nullable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Utility class to encapsulate the logic of building an {@link ExecutionGraph} from a {@link JobGraph}.
+ */
+public class ExecutionGraphBuilder {
+
+ /**
+ * Builds the ExecutionGraph from the JobGraph.
+ * If a prior execution graph exists, the JobGraph will be attached. If no prior execution
+ * graph exists, then the JobGraph will become attach to a new emoty execution graph.
+ */
+ public static ExecutionGraph buildGraph(
+ @Nullable ExecutionGraph prior,
+ JobGraph jobGraph,
+ Configuration jobManagerConfig,
+ Executor executor,
+ ClassLoader classLoader,
+ CheckpointRecoveryFactory recoveryFactory,
+ SavepointStore savepointStore,
+ Time timeout,
+ RestartStrategy restartStrategy,
+ MetricGroup metrics,
+ int parallelismForAutoMax,
+ Logger log)
+ throws JobExecutionException, JobException
+ {
+ final ExecutionContext executionContext = ExecutionContext$.MODULE$.fromExecutor(executor);
+
+ return buildGraph(prior, jobGraph, jobManagerConfig, executionContext,
+ classLoader, recoveryFactory, savepointStore, timeout, restartStrategy,
+ metrics, parallelismForAutoMax, log);
+ }
+
+ /**
+ * Builds the ExecutionGraph from the JobGraph.
+ * If a prior execution graph exists, the JobGraph will be attached. If no prior execution
+ * graph exists, then the JobGraph will become attach to a new emoty execution graph.
+ */
+ public static ExecutionGraph buildGraph(
+ @Nullable ExecutionGraph prior,
+ JobGraph jobGraph,
+ Configuration jobManagerConfig,
+ ExecutionContext executionContext,
+ ClassLoader classLoader,
+ CheckpointRecoveryFactory recoveryFactory,
+ SavepointStore savepointStore,
+ Time timeout,
+ RestartStrategy restartStrategy,
+ MetricGroup metrics,
+ int parallelismForAutoMax,
+ Logger log)
+ throws JobExecutionException, JobException
+ {
+ checkNotNull(jobGraph, "job graph cannot be null");
+
+ final String jobName = jobGraph.getName();
+ final JobID jobId = jobGraph.getJobID();
+
+ // create a new execution graph, if none exists so far
+ final ExecutionGraph executionGraph = (prior != null) ? prior :
+ new ExecutionGraph(
+ executionContext,
+ jobId,
+ jobName,
+ jobGraph.getJobConfiguration(),
+ jobGraph.getSerializedExecutionConfig(),
+ new FiniteDuration(timeout.getSize(), timeout.getUnit()),
+ restartStrategy,
+ jobGraph.getUserJarBlobKeys(),
+ jobGraph.getClasspaths(),
+ classLoader,
+ metrics);
+
+ // set the basic properties
+
+ executionGraph.setScheduleMode(jobGraph.getScheduleMode());
+ executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
+
+ try {
+ executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
+ }
+ catch (Throwable t) {
+ log.warn("Cannot create JSON plan for job", t);
+ // give the graph an empty plan
+ executionGraph.setJsonPlan("{}");
+ }
+
+ // initialize the vertices that have a master initialization hook
+ // file output formats create directories here, input formats create splits
+
+ final long initMasterStart = System.nanoTime();
+ log.info("Running initialization on master for job {} ({}).", jobName, jobId);
+
+ for (JobVertex vertex : jobGraph.getVertices()) {
+ String executableClass = vertex.getInvokableClassName();
+ if (executableClass == null || executableClass.isEmpty()) {
+ throw new JobSubmissionException(jobId,
+ "The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class.");
+ }
+
+ if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
+ vertex.setParallelism(parallelismForAutoMax);
+ }
+
+ try {
+ vertex.initializeOnMaster(classLoader);
+ }
+ catch (Throwable t) {
+ throw new JobExecutionException(jobId,
+ "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t);
+ }
+ }
+
+ log.info("Successfully ran initialization on master in {} ms.",
+ (System.nanoTime() - initMasterStart) / 1_000_000);
+
+ // topologically sort the job vertices and attach the graph to the existing one
+ List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
+ if (log.isDebugEnabled()) {
+ log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(), jobName, jobId);
+ }
+ executionGraph.attachJobGraph(sortedTopology);
+
+ if (log.isDebugEnabled()) {
+ log.debug("Successfully created execution graph from job graph {} ({}).", jobName, jobId);
+ }
+
+ // configure the state checkpointing
+ JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
+ if (snapshotSettings != null) {
+
+ List<ExecutionJobVertex> triggerVertices =
+ idToVertex(snapshotSettings.getVerticesToTrigger(), executionGraph);
+
+ List<ExecutionJobVertex> ackVertices =
+ idToVertex(snapshotSettings.getVerticesToAcknowledge(), executionGraph);
+
+ List<ExecutionJobVertex> confirmVertices =
+ idToVertex(snapshotSettings.getVerticesToConfirm(), executionGraph);
+
+ CompletedCheckpointStore completedCheckpoints;
+ CheckpointIDCounter checkpointIdCounter;
+ try {
+ completedCheckpoints = recoveryFactory.createCheckpointStore(jobId, classLoader);
+ checkpointIdCounter = recoveryFactory.createCheckpointIDCounter(jobId);
+ }
+ catch (Exception e) {
+ throw new JobExecutionException(jobId, "Failed to initialize high-availability checkpoint handler", e);
+ }
+
+ // Checkpoint stats tracker
+ boolean isStatsDisabled = jobManagerConfig.getBoolean(
+ ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
+ ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
+
+ CheckpointStatsTracker checkpointStatsTracker;
+ if (isStatsDisabled) {
+ checkpointStatsTracker = new DisabledCheckpointStatsTracker();
+ }
+ else {
+ int historySize = jobManagerConfig.getInteger(
+ ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
+ ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
+
+ checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, metrics);
+ }
+
+ executionGraph.enableSnapshotCheckpointing(
+ snapshotSettings.getCheckpointInterval(),
+ snapshotSettings.getCheckpointTimeout(),
+ snapshotSettings.getMinPauseBetweenCheckpoints(),
+ snapshotSettings.getMaxConcurrentCheckpoints(),
+ triggerVertices,
+ ackVertices,
+ confirmVertices,
+ checkpointIdCounter,
+ completedCheckpoints,
+ savepointStore,
+ checkpointStatsTracker);
+ }
+
+ return executionGraph;
+ }
+
+ private static List<ExecutionJobVertex> idToVertex(
+ List<JobVertexID> jobVertices, ExecutionGraph executionGraph) throws IllegalArgumentException {
+
+ List<ExecutionJobVertex> result = new ArrayList<>(jobVertices.size());
+
+ for (JobVertexID id : jobVertices) {
+ ExecutionJobVertex vertex = executionGraph.getJobVertex(id);
+ if (vertex != null) {
+ result.add(vertex);
+ } else {
+ throw new IllegalArgumentException(
+ "The snapshot checkpointing settings refer to non-existent vertex " + id);
+ }
+ }
+
+ return result;
+ }
+
+ // ------------------------------------------------------------------------
+
+ /** This class is not supposed to be instantiated */
+ private ExecutionGraphBuilder() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/05436f4b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
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 01f9cec..e90f2d2 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
@@ -29,7 +29,8 @@ import akka.actor.Status.{Failure, Success}
import akka.actor._
import akka.pattern.ask
import grizzled.slf4j.Logger
-import org.apache.flink.api.common.{ExecutionConfig, JobID}
+import org.apache.flink.api.common.JobID
+import org.apache.flink.api.common.time.Time
import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
import org.apache.flink.core.fs.FileSystem
import org.apache.flink.core.io.InputSplitAssigner
@@ -49,11 +50,10 @@ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceMa
import org.apache.flink.runtime.clusterframework.types.ResourceID
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger}
+import org.apache.flink.runtime.executiongraph.{ExecutionGraphBuilder, ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger}
import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
import org.apache.flink.runtime.io.network.PartitionState
-import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator
-import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
+import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus}
import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore.SubmittedJobGraphListener
import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
import org.apache.flink.runtime.leaderelection.{LeaderContender, LeaderElectionService, StandaloneLeaderElectionService}
@@ -1114,7 +1114,7 @@ class JobManager(
Option(jobGraph.getSerializedExecutionConfig()
.deserializeValue(userCodeLoader)
.getRestartStrategy())
- .map(RestartStrategyFactory.createRestartStrategy(_)) match {
+ .map(RestartStrategyFactory.createRestartStrategy) match {
case Some(strategy) => strategy
case None => restartStrategyFactory.createRestartStrategy()
}
@@ -1131,148 +1131,34 @@ class JobManager(
new UnregisteredMetricsGroup()
}
+ val numSlots = scheduler.getTotalNumberOfSlots()
+
// see if there already exists an ExecutionGraph for the corresponding job ID
- executionGraph = currentJobs.get(jobGraph.getJobID) match {
+ val registerNewGraph = currentJobs.get(jobGraph.getJobID) match {
case Some((graph, currentJobInfo)) =>
+ executionGraph = graph
currentJobInfo.setLastActive()
- graph
+ false
case None =>
- val graph = new ExecutionGraph(
- executionContext,
- jobGraph.getJobID,
- jobGraph.getName,
- jobGraph.getJobConfiguration,
- jobGraph.getSerializedExecutionConfig,
- timeout,
- restartStrategy,
- jobGraph.getUserJarBlobKeys,
- jobGraph.getClasspaths,
- userCodeLoader,
- jobMetrics)
-
- currentJobs.put(jobGraph.getJobID, (graph, jobInfo))
- graph
- }
-
- executionGraph.setScheduleMode(jobGraph.getScheduleMode())
- executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling())
-
- try {
- executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph))
- }
- catch {
- case t: Throwable =>
- 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) {
- log.debug(s"Running initialization on master for job $jobId ($jobName).")
- }
-
- 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)
- }
-
- try {
- vertex.initializeOnMaster(userCodeLoader)
- }
- catch {
- case t: Throwable =>
- throw new JobExecutionException(jobId,
- "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage, t)
- }
+ true
}
- // 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).")
- }
-
- // 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 triggerVertices: java.util.List[ExecutionJobVertex] =
- snapshotSettings.getVerticesToTrigger().asScala.map(idToVertex).asJava
-
- val ackVertices: java.util.List[ExecutionJobVertex] =
- snapshotSettings.getVerticesToAcknowledge().asScala.map(idToVertex).asJava
-
- val confirmVertices: java.util.List[ExecutionJobVertex] =
- snapshotSettings.getVerticesToConfirm().asScala.map(idToVertex).asJava
-
- val completedCheckpoints = checkpointRecoveryFactory
- .createCheckpointStore(jobId, userCodeLoader)
-
- val checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(jobId)
-
- // Checkpoint stats tracker
- val isStatsDisabled: Boolean = flinkConfiguration.getBoolean(
- ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
- ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE)
-
- val checkpointStatsTracker : CheckpointStatsTracker =
- if (isStatsDisabled) {
- new DisabledCheckpointStatsTracker()
- } else {
- val historySize: Int = flinkConfiguration.getInteger(
- ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
- ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE)
-
- new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics)
- }
-
- val jobParallelism = jobGraph.getSerializedExecutionConfig
- .deserializeValue(userCodeLoader).getParallelism()
-
- val parallelism = if (jobParallelism == ExecutionConfig.PARALLELISM_AUTO_MAX) {
- numSlots
- } else {
- jobParallelism
- }
-
- executionGraph.enableSnapshotCheckpointing(
- snapshotSettings.getCheckpointInterval,
- snapshotSettings.getCheckpointTimeout,
- snapshotSettings.getMinPauseBetweenCheckpoints,
- snapshotSettings.getMaxConcurrentCheckpoints,
- triggerVertices,
- ackVertices,
- confirmVertices,
- checkpointIdCounter,
- completedCheckpoints,
- savepointStore,
- checkpointStatsTracker)
+ executionGraph = ExecutionGraphBuilder.buildGraph(
+ executionGraph,
+ jobGraph,
+ flinkConfiguration,
+ executionContext,
+ userCodeLoader,
+ checkpointRecoveryFactory,
+ savepointStore,
+ Time.of(timeout.length, timeout.unit),
+ restartStrategy,
+ jobMetrics,
+ numSlots,
+ log.logger)
+
+ if (registerNewGraph) {
+ currentJobs.put(jobGraph.getJobID, (executionGraph, jobInfo))
}
// get notified about job status changes
[4/8] flink git commit: [FLINK-4768] [core] Migrate high-availability
configuration parameters to ConfigOptions
Posted by se...@apache.org.
[FLINK-4768] [core] Migrate high-availability configuration parameters to ConfigOptions
This closes #2607
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c8dc074a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c8dc074a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c8dc074a
Branch: refs/heads/flip-6
Commit: c8dc074a1899fa0f7d6ce7c6377c5e3d30159c18
Parents: d71a09c
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Oct 8 01:41:02 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200
----------------------------------------------------------------------
.../org/apache/flink/client/cli/DefaultCLI.java | 5 +-
.../configuration/HighAvailabilityOptions.java | 139 +++++++++++++++++++
.../webmonitor/WebRuntimeMonitorITCase.java | 7 +-
.../flink/runtime/blob/FileSystemBlobStore.java | 22 ++-
.../jobmanager/HighAvailabilityMode.java | 8 +-
.../flink/runtime/security/SecurityContext.java | 11 +-
.../flink/runtime/util/ZooKeeperUtils.java | 68 +++------
.../zookeeper/FlinkZooKeeperQuorumPeer.java | 46 +++---
.../flink/runtime/jobmanager/JobManager.scala | 14 +-
.../flink/runtime/blob/BlobRecoveryITCase.java | 5 +-
.../BlobLibraryCacheRecoveryITCase.java | 5 +-
.../jobmanager/HighAvailabilityModeTest.java | 13 +-
.../jobmanager/JobManagerHARecoveryTest.java | 5 +-
.../ZooKeeperLeaderElectionTest.java | 25 ++--
.../ZooKeeperLeaderRetrievalTest.java | 15 +-
.../runtime/testutils/ZooKeeperTestUtils.java | 13 +-
.../flink/runtime/util/ZooKeeperUtilTest.java | 3 +-
.../zookeeper/ZooKeeperTestEnvironment.java | 10 +-
.../runtime/testingUtils/TestingUtils.scala | 13 +-
.../connectors/fs/RollingSinkSecuredITCase.java | 5 +-
.../flink/test/util/SecureTestEnvironment.java | 3 +-
.../apache/flink/test/util/TestBaseUtils.java | 3 +-
.../flink/test/recovery/ChaosMonkeyITCase.java | 3 +-
...agerHAProcessFailureBatchRecoveryITCase.java | 5 +-
...CliFrontendYarnAddressConfigurationTest.java | 11 +-
.../flink/yarn/YARNHighAvailabilityITCase.java | 3 +-
.../yarn/AbstractYarnClusterDescriptor.java | 5 +-
.../flink/yarn/YarnApplicationMasterRunner.java | 3 +-
.../flink/yarn/cli/FlinkYarnSessionCli.java | 6 +-
29 files changed, 302 insertions(+), 172 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
index 18fa323..8f79403 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java
@@ -19,11 +19,12 @@ package org.apache.flink.client.cli;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Options;
+
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.deployment.StandaloneClusterDescriptor;
import org.apache.flink.client.program.StandaloneClusterClient;
-import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import java.net.InetSocketAddress;
@@ -64,7 +65,7 @@ public class DefaultCLI implements CustomCommandLine<StandaloneClusterClient> {
if (commandLine.hasOption(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt())) {
String zkNamespace = commandLine.getOptionValue(CliFrontendParser.ZOOKEEPER_NAMESPACE_OPTION.getOpt());
- config.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+ config.setString(HighAvailabilityOptions.HA_CLUSTER_ID.key(), zkNamespace);
}
StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
new file mode 100644
index 0000000..1ee988a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java
@@ -0,0 +1,139 @@
+/*
+ * 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.configuration;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/**
+ * The set of configuration options relating to high-availability settings.
+ */
+@PublicEvolving
+public class HighAvailabilityOptions {
+
+ // ------------------------------------------------------------------------
+ // Required High Availability Options
+ // ------------------------------------------------------------------------
+
+ /**
+ * Defines high-availability mode used for the cluster execution.
+ * A value of "NONE" signals no highly available setup.
+ * To enable high-availability, set this mode to "ZOOKEEPER".
+ */
+ public static final ConfigOption<String> HA_MODE =
+ key("high-availability")
+ .defaultValue("NONE")
+ .withDeprecatedKeys("recovery.mode");
+
+ /**
+ * The ID of the Flink cluster, used to separate multiple Flink clusters
+ * Needs to be set for standalone clusters, is automatically inferred in YARN and Mesos.
+ */
+ public static final ConfigOption<String> HA_CLUSTER_ID =
+ key("high-availability.cluster-id")
+ .defaultValue("/default")
+ .withDeprecatedKeys("high-availability.zookeeper.path.namespace", "recovery.zookeeper.path.namespace");
+
+ /**
+ * File system path (URI) where Flink persists metadata in high-availability setups
+ */
+ public static final ConfigOption<String> HA_STORAGE_PATH =
+ key("high-availability.storageDir")
+ .noDefaultValue()
+ .withDeprecatedKeys("high-availability.zookeeper.storageDir", "recovery.zookeeper.storageDir");
+
+ /**
+ * The ZooKeeper quorum to use, when running Flink in a high-availability mode with ZooKeeper.
+ */
+ public static final ConfigOption<String> HA_ZOOKEEPER_QUORUM =
+ key("high-availability.zookeeper.quorum")
+ .noDefaultValue()
+ .withDeprecatedKeys("recovery.zookeeper.quorum");
+
+
+ // ------------------------------------------------------------------------
+ // Recovery Options
+ // ------------------------------------------------------------------------
+
+ /**
+ * Optional port (range) used by the job manager in high-availability mode.
+ */
+ public static final ConfigOption<String> HA_JOB_MANAGER_PORT_RANGE =
+ key("high-availability.jobmanager.port")
+ .defaultValue("0")
+ .withDeprecatedKeys("recovery.jobmanager.port");
+
+ /**
+ * The time before a JobManager after a fail over recovers the current jobs.
+ */
+ public static final ConfigOption<String> HA_JOB_DELAY =
+ key("high-availability.job.delay")
+ .noDefaultValue()
+ .withDeprecatedKeys("recovery.job.delay");
+
+ // ------------------------------------------------------------------------
+ // ZooKeeper Options
+ // ------------------------------------------------------------------------
+
+ /**
+ * The root path under which Flink stores its entries in ZooKeeper
+ */
+ public static final ConfigOption<String> HA_ZOOKEEPER_ROOT =
+ key("high-availability.zookeeper.path.root")
+ .defaultValue("/flink")
+ .withDeprecatedKeys("recovery.zookeeper.path.root");
+
+ // ------------------------------------------------------------------------
+ // ZooKeeper Client Settings
+ // ------------------------------------------------------------------------
+
+ public static final ConfigOption<Integer> ZOOKEEPER_SESSION_TIMEOUT =
+ key("high-availability.zookeeper.client.session-timeout")
+ .defaultValue(60000)
+ .withDeprecatedKeys("recovery.zookeeper.client.session-timeout");
+
+ public static final ConfigOption<Integer> ZOOKEEPER_CONNECTION_TIMEOUT =
+ key("high-availability.zookeeper.client.connection-timeout")
+ .defaultValue(15000)
+ .withDeprecatedKeys("recovery.zookeeper.client.connection-timeout");
+
+ public static final ConfigOption<Integer> ZOOKEEPER_RETRY_WAIT =
+ key("high-availability.zookeeper.client.retry-wait")
+ .defaultValue(5000)
+ .withDeprecatedKeys("recovery.zookeeper.client.retry-wait");
+
+ public static final ConfigOption<Integer> ZOOKEEPER_MAX_RETRY_ATTEMPTS =
+ key("high-availability.zookeeper.client.max-retry-attempts")
+ .defaultValue(3)
+ .withDeprecatedKeys("recovery.zookeeper.client.max-retry-attempts");
+
+ public static final ConfigOption<Boolean> ZOOKEEPER_SASL_DISABLE =
+ key("zookeeper.sasl.disable")
+ .defaultValue(true);
+
+ public static final ConfigOption<String> ZOOKEEPER_SASL_SERVICE_NAME =
+ key("zookeeper.sasl.service-name")
+ .noDefaultValue();
+
+ // ------------------------------------------------------------------------
+
+ /** Not intended to be instantiated */
+ private HighAvailabilityOptions() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
index 54c5e76..1ae776c 100644
--- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
+++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java
@@ -24,6 +24,7 @@ import io.netty.handler.codec.http.HttpResponseStatus;
import org.apache.curator.test.TestingServer;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.jobmanager.JobManager;
import org.apache.flink.runtime.jobmanager.MemoryArchivist;
@@ -237,7 +238,7 @@ public class WebRuntimeMonitorITCase extends TestLogger {
followingClient.sendGetRequest("index.html", deadline.timeLeft());
response = followingClient.getNextResponse(deadline.timeLeft());
assertEquals(HttpResponseStatus.TEMPORARY_REDIRECT, response.getStatus());
- assertTrue(response.getLocation().contains("" + leadingWebMonitor.getServerPort()));
+ assertTrue(response.getLocation().contains(String.valueOf(leadingWebMonitor.getServerPort())));
// Kill the leader
leadingSystem.shutdown();
@@ -296,8 +297,8 @@ public class WebRuntimeMonitorITCase extends TestLogger {
final Configuration config = new Configuration();
config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, 0);
config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.toString());
- config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zooKeeper.getConnectString());
+ config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeper.getConnectString());
actorSystem = AkkaUtils.createDefaultActorSystem();
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java
----------------------------------------------------------------------
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
index ee189d4..deba738 100644
--- 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
@@ -19,14 +19,17 @@
package org.apache.flink.runtime.blob;
import com.google.common.io.Files;
+
+import org.apache.commons.lang3.StringUtils;
+
import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.ConfigurationUtil;
import org.apache.flink.util.IOUtils;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -52,16 +55,11 @@ class FileSystemBlobStore implements BlobStore {
private final String basePath;
FileSystemBlobStore(Configuration config) throws IOException {
- String storagePath = ConfigurationUtil.getStringWithDeprecatedKeys(
- config,
- ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH,
- null,
- ConfigConstants.ZOOKEEPER_RECOVERY_PATH);
-
- if (storagePath == null) {
- throw new IllegalConfigurationException(String.format("Missing configuration for " +
- "ZooKeeper file system path. Please specify via " +
- "'%s' key.", ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH));
+ String storagePath = config.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
+
+ if (storagePath == null || StringUtils.isBlank(storagePath)) {
+ throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
+ " Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
}
this.basePath = storagePath + "/blob";
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
index 087ad3b..fa2db48 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HighAvailabilityMode.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.jobmanager;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.ConfigurationUtil;
+import org.apache.flink.configuration.HighAvailabilityOptions;
/**
* High availability mode for Flink's cluster execution. Currently supported modes are:
@@ -43,11 +43,7 @@ public enum HighAvailabilityMode {
* configured.
*/
public static HighAvailabilityMode fromConfig(Configuration config) {
- String haMode = ConfigurationUtil.getStringWithDeprecatedKeys(
- config,
- ConfigConstants.HA_MODE,
- null,
- ConfigConstants.RECOVERY_MODE);
+ String haMode = config.getValue(HighAvailabilityOptions.HA_MODE);
if (haMode == null) {
return HighAvailabilityMode.NONE;
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
index be6611f..67dd78c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
@@ -23,6 +23,7 @@ import org.apache.flink.annotation.Internal;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.util.Preconditions;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
@@ -182,9 +183,9 @@ public class SecurityContext {
//with pseudo JAAS configuration file if SASL auth is enabled for ZK
System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, "");
- boolean disableSaslClient = configuration.getBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE,
- ConfigConstants.DEFAULT_ZOOKEEPER_SASL_DISABLE);
- if(disableSaslClient) {
+ boolean disableSaslClient = configuration.getBoolean(HighAvailabilityOptions.ZOOKEEPER_SASL_DISABLE);
+
+ if (disableSaslClient) {
LOG.info("SASL client auth for ZK will be disabled");
//SASL auth is disabled by default but will be enabled if specified in configuration
System.setProperty(ZOOKEEPER_SASL_CLIENT,"false");
@@ -212,8 +213,8 @@ public class SecurityContext {
System.setProperty(JAVA_SECURITY_AUTH_LOGIN_CONFIG, jaasConfFile.getAbsolutePath());
System.setProperty(ZOOKEEPER_SASL_CLIENT, "true");
- String zkSaslServiceName = configuration.getString(ConfigConstants.ZOOKEEPER_SASL_SERVICE_NAME, null);
- if(!StringUtils.isBlank(zkSaslServiceName)) {
+ String zkSaslServiceName = configuration.getValue(HighAvailabilityOptions.ZOOKEEPER_SASL_SERVICE_NAME);
+ if (!StringUtils.isBlank(zkSaslServiceName)) {
LOG.info("ZK SASL service name: {} is provided in the configuration", zkSaslServiceName);
System.setProperty(ZOOKEEPER_SASL_CLIENT_USERNAME,zkSaslServiceName);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
----------------------------------------------------------------------
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 5e69875..137a85b 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
@@ -18,12 +18,14 @@
package org.apache.flink.runtime.util;
+import org.apache.commons.lang3.StringUtils;
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.HighAvailabilityOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
@@ -57,53 +59,25 @@ public class ZooKeeperUtils {
* @return {@link CuratorFramework} instance
*/
public static CuratorFramework startCuratorFramework(Configuration configuration) {
- String zkQuorum = ConfigurationUtil.getStringWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
- null,
- ConfigConstants.ZOOKEEPER_QUORUM_KEY);
+ String zkQuorum = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM);
- if (zkQuorum == null || zkQuorum.equals("")) {
+ if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
throw new RuntimeException("No valid ZooKeeper quorum has been specified. " +
"You can specify the quorum via the configuration key '" +
- ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY + "'.");
+ HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM.key() + "'.");
}
- int sessionTimeout = ConfigurationUtil.getIntegerWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_SESSION_TIMEOUT,
- ConfigConstants.DEFAULT_ZOOKEEPER_SESSION_TIMEOUT,
- ConfigConstants.ZOOKEEPER_SESSION_TIMEOUT);
+ int sessionTimeout = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT);
- int connectionTimeout = ConfigurationUtil.getIntegerWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_CONNECTION_TIMEOUT,
- ConfigConstants.DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT,
- ConfigConstants.ZOOKEEPER_CONNECTION_TIMEOUT);
+ int connectionTimeout = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_CONNECTION_TIMEOUT);
- int retryWait = ConfigurationUtil.getIntegerWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_RETRY_WAIT,
- ConfigConstants.DEFAULT_ZOOKEEPER_RETRY_WAIT,
- ConfigConstants.ZOOKEEPER_RETRY_WAIT);
+ int retryWait = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_RETRY_WAIT);
- int maxRetryAttempts = ConfigurationUtil.getIntegerWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_MAX_RETRY_ATTEMPTS,
- ConfigConstants.DEFAULT_ZOOKEEPER_MAX_RETRY_ATTEMPTS,
- ConfigConstants.ZOOKEEPER_MAX_RETRY_ATTEMPTS);
+ int maxRetryAttempts = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_MAX_RETRY_ATTEMPTS);
- String root = ConfigurationUtil.getStringWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_DIR_KEY,
- ConfigConstants.DEFAULT_ZOOKEEPER_DIR_KEY,
- ConfigConstants.ZOOKEEPER_DIR_KEY);
+ String root = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT);
- String namespace = ConfigurationUtil.getStringWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY,
- ConfigConstants.DEFAULT_ZOOKEEPER_NAMESPACE_KEY,
- ConfigConstants.ZOOKEEPER_NAMESPACE_KEY);
+ String namespace = configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
String rootWithNamespace = generateZookeeperPath(root, namespace);
@@ -138,13 +112,9 @@ public class ZooKeeperUtils {
public static String getZooKeeperEnsemble(Configuration flinkConf)
throws IllegalConfigurationException {
- String zkQuorum = ConfigurationUtil.getStringWithDeprecatedKeys(
- flinkConf,
- ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
- "",
- ConfigConstants.ZOOKEEPER_QUORUM_KEY);
+ String zkQuorum = flinkConf.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM);
- if (zkQuorum == null || zkQuorum.equals("")) {
+ if (zkQuorum == null || StringUtils.isBlank(zkQuorum)) {
throw new IllegalConfigurationException("No ZooKeeper quorum specified in config.");
}
@@ -367,15 +337,11 @@ public class ZooKeeperUtils {
Configuration configuration,
String prefix) throws IOException {
- String rootPath = ConfigurationUtil.getStringWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH,
- "",
- ConfigConstants.ZOOKEEPER_RECOVERY_PATH);
+ String rootPath = configuration.getValue(HighAvailabilityOptions.HA_STORAGE_PATH);
- if (rootPath.equals("")) {
- throw new IllegalConfigurationException("Missing recovery path. Specify via " +
- "configuration key '" + ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH + "'.");
+ if (rootPath == null || StringUtils.isBlank(rootPath)) {
+ throw new IllegalConfigurationException("Missing high-availability storage path for metadata." +
+ " Specify via configuration key '" + HighAvailabilityOptions.HA_STORAGE_PATH + "'.");
} else {
return new FileSystemStateStorageHelper<T>(rootPath, prefix);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
index 9fba529..c4140c0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/FlinkZooKeeperQuorumPeer.java
@@ -19,9 +19,9 @@
package org.apache.flink.runtime.zookeeper;
import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.runtime.util.EnvironmentInformation;
+
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.ZooKeeperServerMain;
@@ -47,8 +47,25 @@ import java.util.UUID;
*/
public class FlinkZooKeeperQuorumPeer {
+ /** ZooKeeper default client port. */
+ public static final int DEFAULT_ZOOKEEPER_CLIENT_PORT = 2181;
+
+ /** ZooKeeper default init limit. */
+ public static final int DEFAULT_ZOOKEEPER_INIT_LIMIT = 10;
+
+ /** ZooKeeper default sync limit. */
+ public static final int DEFAULT_ZOOKEEPER_SYNC_LIMIT = 5;
+
+ /** ZooKeeper default peer port. */
+ public static final int DEFAULT_ZOOKEEPER_PEER_PORT = 2888;
+
+ /** ZooKeeper default leader port. */
+ public static final int DEFAULT_ZOOKEEPER_LEADER_PORT = 3888;
+
private static final Logger LOG = LoggerFactory.getLogger(FlinkZooKeeperQuorumPeer.class);
+ // ------------------------------------------------------------------------
+
public static void main(String[] args) {
try {
// startup checks and logging
@@ -67,6 +84,8 @@ public class FlinkZooKeeperQuorumPeer {
}
}
+ // ------------------------------------------------------------------------
+
/**
* Runs a ZooKeeper {@link QuorumPeer} if further peers are configured or a single
* {@link ZooKeeperServer} if no further peers are configured.
@@ -120,26 +139,23 @@ public class FlinkZooKeeperQuorumPeer {
private static void setRequiredProperties(Properties zkProps) {
// Set default client port
if (zkProps.getProperty("clientPort") == null) {
- int clientPort = ConfigConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT;
- zkProps.setProperty("clientPort", String.valueOf(clientPort));
+ zkProps.setProperty("clientPort", String.valueOf(DEFAULT_ZOOKEEPER_CLIENT_PORT));
- LOG.warn("No 'clientPort' configured. Set to '{}'.", clientPort);
+ LOG.warn("No 'clientPort' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_CLIENT_PORT);
}
// Set default init limit
if (zkProps.getProperty("initLimit") == null) {
- int initLimit = ConfigConstants.DEFAULT_ZOOKEEPER_INIT_LIMIT;
- zkProps.setProperty("initLimit", String.valueOf(initLimit));
+ zkProps.setProperty("initLimit", String.valueOf(DEFAULT_ZOOKEEPER_INIT_LIMIT));
- LOG.warn("No 'initLimit' configured. Set to '{}'.", initLimit);
+ LOG.warn("No 'initLimit' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_INIT_LIMIT);
}
// Set default sync limit
if (zkProps.getProperty("syncLimit") == null) {
- int syncLimit = ConfigConstants.DEFAULT_ZOOKEEPER_SYNC_LIMIT;
- zkProps.setProperty("syncLimit", String.valueOf(syncLimit));
+ zkProps.setProperty("syncLimit", String.valueOf(DEFAULT_ZOOKEEPER_SYNC_LIMIT));
- LOG.warn("No 'syncLimit' configured. Set to '{}'.", syncLimit);
+ LOG.warn("No 'syncLimit' configured. Set to '{}'.", DEFAULT_ZOOKEEPER_SYNC_LIMIT);
}
// Set default data dir
@@ -152,8 +168,8 @@ public class FlinkZooKeeperQuorumPeer {
LOG.warn("No 'dataDir' configured. Set to '{}'.", dataDir);
}
- int peerPort = ConfigConstants.DEFAULT_ZOOKEEPER_PEER_PORT;
- int leaderPort = ConfigConstants.DEFAULT_ZOOKEEPER_LEADER_PORT;
+ int peerPort = DEFAULT_ZOOKEEPER_PEER_PORT;
+ int leaderPort = DEFAULT_ZOOKEEPER_LEADER_PORT;
// Set peer and leader ports if none given, because ZooKeeper complains if multiple
// servers are configured, but no ports are given.
@@ -220,12 +236,8 @@ public class FlinkZooKeeperQuorumPeer {
// Write myid to file. We use a File Writer, because that properly propagates errors,
// while the PrintWriter swallows errors
- FileWriter writer = new FileWriter(new File(dataDir, "myid"));
- try {
+ try (FileWriter writer = new FileWriter(new File(dataDir, "myid"))) {
writer.write(String.valueOf(id));
}
- finally {
- writer.close();
- }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
----------------------------------------------------------------------
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 e90f2d2..be820ae 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
@@ -31,7 +31,7 @@ import akka.pattern.ask
import grizzled.slf4j.Logger
import org.apache.flink.api.common.JobID
import org.apache.flink.api.common.time.Time
-import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration}
+import org.apache.flink.configuration.{ConfigConstants, Configuration, GlobalConfiguration, HighAvailabilityOptions}
import org.apache.flink.core.fs.FileSystem
import org.apache.flink.core.io.InputSplitAssigner
import org.apache.flink.metrics.{Gauge, MetricGroup}
@@ -2367,9 +2367,7 @@ object JobManager {
configuration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
// The port range of allowed job manager ports or 0 for random
- configuration.getString(
- ConfigConstants.RECOVERY_JOB_MANAGER_PORT,
- ConfigConstants.DEFAULT_HA_JOB_MANAGER_PORT)
+ configuration.getValue(HighAvailabilityOptions.HA_JOB_MANAGER_PORT_RANGE)
}
else {
LOG.info("Starting JobManager without high-availability")
@@ -2501,11 +2499,7 @@ object JobManager {
val savepointStore = SavepointStoreFactory.createFromConfig(configuration)
- val jobRecoveryTimeoutStr = ConfigurationUtil.getStringWithDeprecatedKeys(
- configuration,
- ConfigConstants.HA_JOB_DELAY,
- null,
- ConfigConstants.RECOVERY_JOB_DELAY)
+ val jobRecoveryTimeoutStr = configuration.getValue(HighAvailabilityOptions.HA_JOB_DELAY)
val jobRecoveryTimeout = if (jobRecoveryTimeoutStr == null || jobRecoveryTimeoutStr.isEmpty) {
timeout
@@ -2515,7 +2509,7 @@ object JobManager {
} catch {
case n: NumberFormatException =>
throw new Exception(
- s"Invalid config value for ${ConfigConstants.HA_JOB_DELAY}: " +
+ s"Invalid config value for ${HighAvailabilityOptions.HA_JOB_DELAY.key()}: " +
s"$jobRecoveryTimeoutStr. Value must be a valid duration (such as '10 s' or '1 min')")
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java
----------------------------------------------------------------------
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
index 8464d68..8ba20c9 100644
--- 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
@@ -22,6 +22,7 @@ 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.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.junit.After;
import org.junit.Before;
@@ -68,9 +69,9 @@ public class BlobRecoveryITCase {
try {
Configuration config = new Configuration();
- config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
+ config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
- config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, recoveryDir.getPath());
+ config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, recoveryDir.getPath());
for (int i = 0; i < server.length; i++) {
server[i] = new BlobServer(config);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java
----------------------------------------------------------------------
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
index f6bed56..f6cdf09 100644
--- 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
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.execution.librarycache;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.blob.BlobCache;
import org.apache.flink.runtime.blob.BlobClient;
import org.apache.flink.runtime.blob.BlobKey;
@@ -63,9 +64,9 @@ public class BlobLibraryCacheRecoveryITCase {
try {
Configuration config = new Configuration();
- config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
+ config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
- config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
+ config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.getRoot().getAbsolutePath());
for (int i = 0; i < server.length; i++) {
server[i] = new BlobServer(config);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
index 04c0e48..91fb514 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/HighAvailabilityModeTest.java
@@ -20,7 +20,8 @@ package org.apache.flink.runtime.jobmanager;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+
import org.junit.Test;
import static org.junit.Assert.assertEquals;
@@ -42,7 +43,7 @@ public class HighAvailabilityModeTest {
assertEquals(DEFAULT_HA_MODE, HighAvailabilityMode.fromConfig(config));
// Check not equals default
- config.setString(ConfigConstants.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+ config.setString(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
assertEquals(HighAvailabilityMode.ZOOKEEPER, HighAvailabilityMode.fromConfig(config));
}
@@ -54,16 +55,16 @@ public class HighAvailabilityModeTest {
Configuration config = new Configuration();
// Check mapping of old default to new default
- config.setString(ConfigConstants.RECOVERY_MODE, ConfigConstants.DEFAULT_RECOVERY_MODE);
+ config.setString("recovery.mode", ConfigConstants.DEFAULT_RECOVERY_MODE);
assertEquals(DEFAULT_HA_MODE, HighAvailabilityMode.fromConfig(config));
// Check deprecated config
- config.setString(ConfigConstants.RECOVERY_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+ config.setString("recovery.mode", HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
assertEquals(HighAvailabilityMode.ZOOKEEPER, HighAvailabilityMode.fromConfig(config));
// Check precedence over deprecated config
- config.setString(ConfigConstants.HA_MODE, HighAvailabilityMode.NONE.name().toLowerCase());
- config.setString(ConfigConstants.RECOVERY_MODE, HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
+ config.setString("high-availability", HighAvailabilityMode.NONE.name().toLowerCase());
+ config.setString("recovery.mode", HighAvailabilityMode.ZOOKEEPER.name().toLowerCase());
assertEquals(HighAvailabilityMode.NONE, HighAvailabilityMode.fromConfig(config));
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
index 360588d..5b12eee 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java
@@ -26,6 +26,7 @@ import akka.testkit.JavaTestKit;
import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.akka.ListeningBehaviour;
@@ -131,8 +132,8 @@ public class JobManagerHARecoveryTest {
ActorRef jobManager = null;
ActorRef taskManager = null;
- flinkConfiguration.setString(ConfigConstants.HA_MODE, "zookeeper");
- flinkConfiguration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, temporaryFolder.newFolder().toString());
+ flinkConfiguration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+ flinkConfiguration.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString());
flinkConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slots);
try {
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
index e20985b..1f1eb62 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java
@@ -27,6 +27,7 @@ import org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.curator.test.TestingServer;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.TestLogger;
@@ -89,8 +90,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
@Test
public void testZooKeeperLeaderElectionRetrieval() throws Exception {
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
ZooKeeperLeaderElectionService leaderElectionService = null;
ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
@@ -134,8 +135,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
@Test
public void testZooKeeperReelection() throws Exception {
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
@@ -217,8 +218,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
@Test
public void testZooKeeperReelectionWithReplacement() throws Exception {
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
int num = 3;
int numTries = 30;
@@ -295,8 +296,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
final String leaderPath = "/leader";
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
configuration.setString(ConfigConstants.HA_ZOOKEEPER_LEADER_PATH, leaderPath);
ZooKeeperLeaderElectionService leaderElectionService = null;
@@ -379,8 +380,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
@Test
public void testExceptionForwarding() throws Exception {
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
ZooKeeperLeaderElectionService leaderElectionService = null;
ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
@@ -448,8 +449,8 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
@Test
public void testEphemeralZooKeeperNodes() throws Exception {
Configuration configuration = new Configuration();
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
- configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
+ configuration.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
+ configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
ZooKeeperLeaderElectionService leaderElectionService;
ZooKeeperLeaderRetrievalService leaderRetrievalService = null;
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderRetrievalTest.java
----------------------------------------------------------------------
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 0fe0644..70b1da0 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
@@ -20,16 +20,19 @@ 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.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.jobmanager.JobManager;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
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;
import org.junit.Test;
+
import scala.Option;
import scala.concurrent.duration.FiniteDuration;
@@ -82,8 +85,8 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
long sleepingTime = 1000;
- config.setString(ConfigConstants.HA_MODE, "zookeeper");
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
+ config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
LeaderElectionService leaderElectionService = null;
LeaderElectionService faultyLeaderElectionService;
@@ -179,8 +182,8 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
@Test
public void testTimeoutOfFindConnectingAddress() throws Exception {
Configuration config = new Configuration();
- config.setString(ConfigConstants.HA_MODE, "zookeeper");
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, testingServer.getConnectString());
+ config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString());
FiniteDuration timeout = new FiniteDuration(10, TimeUnit.SECONDS);
@@ -190,7 +193,7 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger{
assertEquals(InetAddress.getLocalHost(), result);
}
- class FindConnectingAddress implements Runnable {
+ static class FindConnectingAddress implements Runnable {
private final Configuration config;
private final FiniteDuration timeout;
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ZooKeeperTestUtils.java
----------------------------------------------------------------------
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
index 7dd7067..07cec32 100644
--- 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
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.testutils;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
@@ -66,8 +67,8 @@ public class ZooKeeperTestUtils {
config.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1);
// ZooKeeper recovery mode
- config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zooKeeperQuorum);
+ config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zooKeeperQuorum);
int connTimeout = 5000;
if (System.getenv().containsKey("CI")) {
@@ -75,20 +76,20 @@ public class ZooKeeperTestUtils {
connTimeout = 30000;
}
- config.setInteger(ConfigConstants.HA_ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout);
- config.setInteger(ConfigConstants.HA_ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
+ config.setInteger(HighAvailabilityOptions.ZOOKEEPER_CONNECTION_TIMEOUT, connTimeout);
+ config.setInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT, connTimeout);
// File system state backend
config.setString(ConfigConstants.STATE_BACKEND, "FILESYSTEM");
config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY, fsStateHandlePath + "/checkpoints");
- config.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, fsStateHandlePath + "/recovery");
+ config.setString(HighAvailabilityOptions.HA_STORAGE_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.AKKA_ASK_TIMEOUT, "100 s");
- config.setString(ConfigConstants.HA_JOB_DELAY, "10 s");
+ config.setString(HighAvailabilityOptions.HA_JOB_DELAY, "10 s");
return config;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
index daed4a4..d5895ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ZooKeeperUtilTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.util;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
@@ -71,7 +72,7 @@ public class ZooKeeperUtilTest extends TestLogger {
}
private Configuration setQuorum(Configuration conf, String quorum) {
- conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, quorum);
+ conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, quorum);
return conf;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperTestEnvironment.java
----------------------------------------------------------------------
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 bd58515..66c4fac 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
@@ -22,9 +22,11 @@ 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.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.util.ZooKeeperUtils;
+
import org.apache.zookeeper.KeeperException;
import java.util.List;
@@ -58,7 +60,7 @@ public class ZooKeeperTestEnvironment {
zooKeeperServer = new TestingServer(true);
zooKeeperCluster = null;
- conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
+ conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
zooKeeperServer.getConnectString());
}
else {
@@ -67,7 +69,7 @@ public class ZooKeeperTestEnvironment {
zooKeeperCluster.start();
- conf.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
+ conf.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
zooKeeperCluster.getConnectString());
}
@@ -127,7 +129,7 @@ public class ZooKeeperTestEnvironment {
*/
public CuratorFramework createClient() {
Configuration config = new Configuration();
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, getConnectString());
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, getConnectString());
return ZooKeeperUtils.startCuratorFramework(config);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
index e878097..97016e4 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala
@@ -28,12 +28,12 @@ import com.typesafe.config.ConfigFactory
import grizzled.slf4j.Logger
import org.apache.flink.api.common.JobExecutionResult
-import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.configuration.{HighAvailabilityOptions, ConfigConstants, Configuration}
import org.apache.flink.runtime.client.JobClient
import org.apache.flink.runtime.clusterframework.FlinkResourceManager
import org.apache.flink.runtime.jobgraph.JobGraph
import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager}
+import org.apache.flink.runtime.jobmanager.{HighAvailabilityMode, MemoryArchivist, JobManager}
import org.apache.flink.runtime.testutils.TestingResourceManager
import org.apache.flink.runtime.util.LeaderRetrievalUtils
import org.apache.flink.runtime.{LogMessages, LeaderSessionMessageFilter, FlinkActor}
@@ -412,8 +412,7 @@ object TestingUtils {
* @param configuration Configuration to use
* @param jobManagerClass JobManager class to instantiate
* @param prefix The prefix to use for the Actor names
- *
- * @return
+ * @return
*/
def createJobManager(
actorSystem: ActorSystem,
@@ -422,7 +421,8 @@ object TestingUtils {
prefix: String)
: ActorGateway = {
- configuration.setString(ConfigConstants.HA_MODE,
+ configuration.setString(
+ HighAvailabilityOptions.HA_MODE,
ConfigConstants.DEFAULT_HA_MODE)
val (actor, _) = JobManager.startJobManagerActors(
@@ -502,7 +502,8 @@ object TestingUtils {
configuration: Configuration)
: ActorGateway = {
- configuration.setString(ConfigConstants.HA_MODE,
+ configuration.setString(
+ HighAvailabilityOptions.HA_MODE,
ConfigConstants.DEFAULT_HA_MODE)
val actor = FlinkResourceManager.startResourceManagerActors(
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
index 051175a..c005814 100644
--- a/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
+++ b/flink-streaming-connectors/flink-connector-filesystem/src/test/java/org/apache/flink/streaming/connectors/fs/RollingSinkSecuredITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.connectors.fs;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.security.SecurityContext;
import org.apache.flink.streaming.util.TestStreamEnvironment;
import org.apache.flink.test.util.SecureTestEnvironment;
@@ -215,10 +216,10 @@ public class RollingSinkSecuredITCase extends RollingSinkITCase {
config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM);
config.setBoolean(ConfigConstants.LOCAL_START_WEBSERVER, false);
config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
- config.setString(ConfigConstants.RECOVERY_MODE, "zookeeper");
+ config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
config.setString(ConfigConstants.STATE_BACKEND, "filesystem");
config.setString(ConfigConstants.ZOOKEEPER_CHECKPOINTS_PATH, hdfsURI + "/flink/checkpoints");
- config.setString(ConfigConstants.ZOOKEEPER_RECOVERY_PATH, hdfsURI + "/flink/recovery");
+ config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, hdfsURI + "/flink/recovery");
config.setString("state.backend.fs.checkpointdir", hdfsURI + "/flink/checkpoints");
SecureTestEnvironment.populateFlinkSecureConfigurations(config);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
index b5e622b..0250c16 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.util;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.security.SecurityContext;
import org.apache.hadoop.minikdc.MiniKdc;
import org.junit.rules.TemporaryFolder;
@@ -115,7 +116,7 @@ public class SecureTestEnvironment {
Configuration flinkConfig = GlobalConfiguration.loadConfiguration();
flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, testKeytab);
flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, testPrincipal);
- flinkConfig.setBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, false);
+ flinkConfig.setBoolean(HighAvailabilityOptions.ZOOKEEPER_SASL_DISABLE, false);
ctx.setFlinkConfiguration(flinkConfig);
TestingSecurityContext.install(ctx, getClientSecurityConfigurationMap());
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index b774f97..aa5e7d3 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -32,6 +32,7 @@ import org.apache.commons.io.IOUtils;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.messages.TaskManagerMessages;
import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
import org.apache.flink.util.TestLogger;
@@ -121,7 +122,7 @@ public class TestBaseUtils extends TestLogger {
if (startZooKeeper) {
config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, 3);
- config.setString(ConfigConstants.HA_MODE, "zookeeper");
+ config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper");
}
return startCluster(config, singleActorSystem);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-tests/src/test/java/org/apache/flink/test/recovery/ChaosMonkeyITCase.java
----------------------------------------------------------------------
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 cc8ab80..4d10bf1 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
@@ -24,6 +24,7 @@ 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.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.akka.ListeningBehaviour;
import org.apache.flink.runtime.instance.AkkaActorGateway;
@@ -564,7 +565,7 @@ public class ChaosMonkeyITCase extends TestLogger {
fail(fsCheckpoints + " does not exist: " + Arrays.toString(FileStateBackendBasePath.listFiles()));
}
- File fsRecovery = new File(new URI(config.getString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, "")).getPath());
+ File fsRecovery = new File(new URI(config.getString(HighAvailabilityOptions.HA_STORAGE_PATH)).getPath());
LOG.info("Checking " + fsRecovery);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
index 9b0d9de..a51f88b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java
@@ -30,6 +30,7 @@ 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.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.instance.ActorGateway;
@@ -149,8 +150,8 @@ public class JobManagerHAProcessFailureBatchRecoveryITCase extends TestLogger {
*/
public void testJobManagerFailure(String zkQuorum, final File coordinateDir) throws Exception {
Configuration config = new Configuration();
- config.setString(ConfigConstants.HA_MODE, "ZOOKEEPER");
- config.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, zkQuorum);
+ config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+ config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkQuorum);
ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(
"leader", 1, config);
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
index 48ad7f5..4bcde16 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/CliFrontendYarnAddressConfigurationTest.java
@@ -19,6 +19,7 @@
package org.apache.flink.yarn;
import org.apache.commons.cli.CommandLine;
+
import org.apache.flink.client.CliFrontend;
import org.apache.flink.client.cli.CliFrontendParser;
import org.apache.flink.client.cli.CommandLineOptions;
@@ -27,7 +28,7 @@ import org.apache.flink.client.cli.RunOptions;
import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.test.util.TestBaseUtils;
import org.apache.flink.yarn.cli.FlinkYarnSessionCli;
@@ -38,20 +39,20 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.client.api.YarnClient;
import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
import org.apache.hadoop.yarn.exceptions.YarnException;
+
import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
+
import org.mockito.Mockito;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
-import java.lang.reflect.Field;
import java.net.InetSocketAddress;
import java.nio.file.Files;
import java.nio.file.StandardOpenOption;
@@ -202,7 +203,7 @@ public class CliFrontendYarnAddressConfigurationTest {
CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString()});
frontend.retrieveClient(options);
- String zkNs = frontend.getConfiguration().getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, "error");
+ String zkNs = frontend.getConfiguration().getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
Assert.assertTrue(zkNs.matches("application_\\d+_0042"));
}
@@ -216,7 +217,7 @@ public class CliFrontendYarnAddressConfigurationTest {
CliFrontendParser.parseRunCommand(new String[] {"-yid", TEST_YARN_APPLICATION_ID.toString(), "-yz", overrideZkNamespace});
frontend.retrieveClient(options);
- String zkNs = frontend.getConfiguration().getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, "error");
+ String zkNs = frontend.getConfiguration().getValue(HighAvailabilityOptions.HA_CLUSTER_ID);
Assert.assertEquals(overrideZkNamespace, zkNs);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index 9d6ff85..79f790f 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.client.program.ClusterClient;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.instance.ActorGateway;
import org.apache.flink.runtime.instance.AkkaActorGateway;
@@ -119,7 +120,7 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
zkServer.getConnectString() + "@@yarn.application-attempts=" + numberApplicationAttempts +
"@@" + ConfigConstants.STATE_BACKEND + "=FILESYSTEM" +
"@@" + FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY + "=" + fsStateHandlePath + "/checkpoints" +
- "@@" + ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH + "=" + fsStateHandlePath + "/recovery");
+ "@@" + HighAvailabilityOptions.HA_STORAGE_PATH.key() + "=" + fsStateHandlePath + "/recovery");
flinkYarnClient.setConfigurationFilePath(new Path(confDirPath + File.separator + "flink-conf.yaml"));
ClusterClient yarnCluster = null;
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
index 848013c..9481c24 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java
@@ -22,6 +22,7 @@ import org.apache.flink.client.CliFrontend;
import org.apache.flink.client.deployment.ClusterDescriptor;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.flink.runtime.security.SecurityContext;
@@ -539,11 +540,11 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor
// no user specified cli argument for namespace?
if (zkNamespace == null || zkNamespace.isEmpty()) {
// namespace defined in config? else use applicationId as default.
- zkNamespace = flinkConfiguration.getString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, String.valueOf(appId));
+ zkNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId));
setZookeeperNamespace(zkNamespace);
}
- flinkConfiguration.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+ flinkConfiguration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace);
if (HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfiguration)) {
// activate re-execution of failed applications
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index b27876b..10e229e 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -25,6 +25,7 @@ import akka.actor.Props;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.clusterframework.BootstrapTools;
import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
@@ -472,7 +473,7 @@ public class YarnApplicationMasterRunner {
// override zookeeper namespace with user cli argument (if provided)
String cliZKNamespace = ENV.get(YarnConfigKeys.ENV_ZOOKEEPER_NAMESPACE);
if (cliZKNamespace != null && !cliZKNamespace.isEmpty()) {
- configuration.setString(ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY, cliZKNamespace);
+ configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, cliZKNamespace);
}
// if a web monitor shall be started, set the port to random binding
http://git-wip-us.apache.org/repos/asf/flink/blob/c8dc074a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
index d09340c..e4da140 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java
@@ -29,6 +29,7 @@ import org.apache.flink.client.cli.CustomCommandLine;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.runtime.security.SecurityContext;
import org.apache.flink.yarn.AbstractYarnClusterDescriptor;
@@ -60,7 +61,6 @@ import java.util.Map;
import java.util.Properties;
import static org.apache.flink.client.cli.CliFrontendParser.ADDRESS_OPTION;
-import static org.apache.flink.configuration.ConfigConstants.HA_ZOOKEEPER_NAMESPACE_KEY;
/**
* Class handling the command line interface to the YARN session.
@@ -513,8 +513,8 @@ public class FlinkYarnSessionCli implements CustomCommandLine<YarnClusterClient>
if(null != applicationID) {
String zkNamespace = cmdLine.hasOption(ZOOKEEPER_NAMESPACE.getOpt()) ?
cmdLine.getOptionValue(ZOOKEEPER_NAMESPACE.getOpt())
- : config.getString(HA_ZOOKEEPER_NAMESPACE_KEY, applicationID);
- config.setString(HA_ZOOKEEPER_NAMESPACE_KEY, zkNamespace);
+ : config.getString(HighAvailabilityOptions.HA_CLUSTER_ID, applicationID);
+ config.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace);
AbstractYarnClusterDescriptor yarnDescriptor = getClusterDescriptor();
yarnDescriptor.setFlinkConfiguration(config);
[8/8] flink git commit: [FLINK-4339] [cluster management] Implement
Slot Pool core on JobManager side
Posted by se...@apache.org.
[FLINK-4339] [cluster management] Implement Slot Pool core on JobManager side
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/48c936ee
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/48c936ee
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/48c936ee
Branch: refs/heads/flip-6
Commit: 48c936eeda7b265f32bbf9f14d15f78eae15c06f
Parents: 34fef47
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Oct 13 04:59:46 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 17:38:17 2016 +0200
----------------------------------------------------------------------
.../CheckpointCoordinatorGateway.java | 2 -
.../clusterframework/types/ResourceProfile.java | 8 +
.../runtime/clusterframework/types/SlotID.java | 16 +-
.../flink/runtime/instance/SlotDescriptor.java | 161 +++++
.../apache/flink/runtime/instance/SlotPool.java | 675 +++++++++++++++++++
.../apache/flink/runtime/akka/AkkaUtils.scala | 4 +-
.../runtime/instance/AllocatedSlotsTest.java | 135 ++++
.../runtime/instance/AvailableSlotsTest.java | 123 ++++
.../flink/runtime/instance/SlotPoolTest.java | 297 ++++++++
9 files changed, 1411 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
index 196ef5c..fa09123 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java
@@ -23,8 +23,6 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.rpc.RpcGateway;
import org.apache.flink.runtime.state.CheckpointStateHandles;
-import java.util.UUID;
-
public interface CheckpointCoordinatorGateway extends RpcGateway {
void acknowledgeCheckpoint(
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
index fa3aabc..1d8075e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
@@ -70,4 +70,12 @@ public class ResourceProfile implements Serializable {
public boolean isMatching(ResourceProfile required) {
return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB();
}
+
+ @Override
+ public String toString() {
+ return "ResourceProfile{" +
+ "cpuCores=" + cpuCores +
+ ", memoryInMB=" + memoryInMB +
+ '}';
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
index e831a5d..237597b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
@@ -33,11 +33,11 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
private final ResourceID resourceId;
/** The numeric id for single slot */
- private final int slotId;
+ private final int slotNumber;
- public SlotID(ResourceID resourceId, int slotId) {
+ public SlotID(ResourceID resourceId, int slotNumber) {
this.resourceId = checkNotNull(resourceId, "ResourceID must not be null");
- this.slotId = slotId;
+ this.slotNumber = slotNumber;
}
// ------------------------------------------------------------------------
@@ -47,6 +47,10 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
return resourceId;
}
+ public int getSlotNumber() {
+ return slotNumber;
+ }
+
// ------------------------------------------------------------------------
@Override
@@ -60,7 +64,7 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
SlotID slotID = (SlotID) o;
- if (slotId != slotID.slotId) {
+ if (slotNumber != slotID.slotNumber) {
return false;
}
return resourceId.equals(slotID.resourceId);
@@ -69,13 +73,13 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
@Override
public int hashCode() {
int result = resourceId.hashCode();
- result = 31 * result + slotId;
+ result = 31 * result + slotNumber;
return result;
}
@Override
public String toString() {
- return resourceId + "_" + slotId;
+ return resourceId + "_" + slotNumber;
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
new file mode 100644
index 0000000..be7cf96
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotDescriptor.java
@@ -0,0 +1,161 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The description of slots, TaskManagers offer one or more task slots, which define a slice of
+ * their resources. This description will contain some static information about the slot, such
+ * as the location and numeric id of the slot, rpc gateway to communicate with the TaskManager which
+ * owns the slot.
+ */
+public class SlotDescriptor {
+
+ /** The ID of the job this slice belongs to. */
+ private final JobID jobID;
+
+ /** The location information of the TaskManager to which this slot belongs */
+ private final TaskManagerLocation taskManagerLocation;
+
+ /** The number of the slot on which the task is deployed */
+ private final int slotNumber;
+
+ /** The resource profile of the slot provides */
+ private final ResourceProfile resourceProfile;
+
+ /** TEMP until the new RPC is in place: The actor gateway to communicate with the TaskManager */
+ private final ActorGateway taskManagerActorGateway;
+
+ public SlotDescriptor(
+ final JobID jobID,
+ final TaskManagerLocation location,
+ final int slotNumber,
+ final ResourceProfile resourceProfile,
+ final ActorGateway actorGateway)
+ {
+ this.jobID = checkNotNull(jobID);
+ this.taskManagerLocation = checkNotNull(location);
+ this.slotNumber = slotNumber;
+ this.resourceProfile = checkNotNull(resourceProfile);
+ this.taskManagerActorGateway = checkNotNull(actorGateway);
+ }
+
+ public SlotDescriptor(final SlotDescriptor other) {
+ this.jobID = other.jobID;
+ this.taskManagerLocation = other.taskManagerLocation;
+ this.slotNumber = other.slotNumber;
+ this.resourceProfile = other.resourceProfile;
+ this.taskManagerActorGateway = other.taskManagerActorGateway;
+ }
+
+ // TODO - temporary workaround until we have the SlotDesriptor in the Slot
+ public SlotDescriptor(final Slot slot) {
+ this.jobID = slot.getJobID();
+ this.taskManagerLocation = slot.getTaskManagerLocation();
+ this.slotNumber = slot.getRootSlotNumber();
+ this.resourceProfile = new ResourceProfile(0, 0);
+ this.taskManagerActorGateway = slot.getTaskManagerActorGateway();
+ }
+
+ /**
+ * Returns the ID of the job this allocated slot belongs to.
+ *
+ * @return the ID of the job this allocated slot belongs to
+ */
+ public JobID getJobID() {
+ return jobID;
+ }
+
+ /**
+ * Gets the number of the slot.
+ *
+ * @return The number of the slot on the TaskManager.
+ */
+ public int getSlotNumber() {
+ return slotNumber;
+ }
+
+ /**
+ * Gets the resource profile of the slot.
+ *
+ * @return The resource profile of the slot.
+ */
+ public ResourceProfile getResourceProfile() {
+ return resourceProfile;
+ }
+
+ /**
+ * Gets the location info of the TaskManager that offers this slot.
+ *
+ * @return The location info of the TaskManager that offers this slot
+ */
+ public TaskManagerLocation getTaskManagerLocation() {
+ return taskManagerLocation;
+ }
+
+ /**
+ * Gets the actor gateway that can be used to send messages to the TaskManager.
+ * <p>
+ * This method should be removed once the new interface-based RPC abstraction is in place
+ *
+ * @return The actor gateway that can be used to send messages to the TaskManager.
+ */
+ public ActorGateway getTaskManagerActorGateway() {
+ return taskManagerActorGateway;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ SlotDescriptor that = (SlotDescriptor) o;
+
+ if (slotNumber != that.slotNumber) {
+ return false;
+ }
+ if (!jobID.equals(that.jobID)) {
+ return false;
+ }
+ return taskManagerLocation.equals(that.taskManagerLocation);
+
+ }
+
+ @Override
+ public int hashCode() {
+ int result = jobID.hashCode();
+ result = 31 * result + taskManagerLocation.hashCode();
+ result = 31 * result + slotNumber;
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return taskManagerLocation + " - " + slotNumber;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
new file mode 100644
index 0000000..e7857c1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java
@@ -0,0 +1,675 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.concurrent.ApplyFunction;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.jobmanager.slots.SlotOwner;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected;
+import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The slot pool serves slot request issued by Scheduler or ExecutionGraph. It will will attempt to acquire new slots
+ * from the ResourceManager when it cannot serve a slot request. If no ResourceManager is currently available,
+ * or it gets a decline from the ResourceManager, or a request times out, it fails the slot request. The slot pool also
+ * holds all the slots that were offered to it and accepted, and can thus provides registered free slots even if the
+ * ResourceManager is down. The slots will only be released when they are useless, e.g. when the job is fully running
+ * but we still have some free slots.
+ * <p>
+ * All the allocation or the slot offering will be identified by self generated AllocationID, we will use it to
+ * eliminate ambiguities.
+ */
+public class SlotPool implements SlotOwner {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SlotPool.class);
+
+ private final Object lock = new Object();
+
+ /** The executor which is used to execute futures */
+ private final Executor executor;
+
+ /** All registered resources, slots will be accepted and used only if the resource is registered */
+ private final Set<ResourceID> registeredResources;
+
+ /** The book-keeping of all allocated slots */
+ private final AllocatedSlots allocatedSlots;
+
+ /** The book-keeping of all available slots */
+ private final AvailableSlots availableSlots;
+
+ /** All pending requests waiting for slots */
+ private final Map<AllocationID, Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>>> pendingRequests;
+
+ /** Timeout of slot allocation */
+ private final Time timeout;
+
+ /** the leader id of job manager */
+ private UUID jobManagerLeaderId;
+
+ /** The leader id of resource manager */
+ private UUID resourceManagerLeaderId;
+
+ /** The gateway to communicate with resource manager */
+ private ResourceManagerGateway resourceManagerGateway;
+
+ public SlotPool(final Executor executor) {
+ this.executor = executor;
+ this.registeredResources = new HashSet<>();
+ this.allocatedSlots = new AllocatedSlots();
+ this.availableSlots = new AvailableSlots();
+ this.pendingRequests = new HashMap<>();
+ this.timeout = Time.of(5, TimeUnit.SECONDS);
+ }
+
+ public void setJobManagerLeaderId(final UUID jobManagerLeaderId) {
+ this.jobManagerLeaderId = jobManagerLeaderId;
+ }
+
+ // ------------------------------------------------------------------------
+ // Slot Allocation
+ // ------------------------------------------------------------------------
+
+ /**
+ * Try to allocate a simple slot with specified resource profile.
+ *
+ * @param jobID The job id which the slot allocated for
+ * @param resourceProfile The needed resource profile
+ * @return The future of allocated simple slot
+ */
+ public Future<SimpleSlot> allocateSimpleSlot(final JobID jobID, final ResourceProfile resourceProfile) {
+ return allocateSimpleSlot(jobID, resourceProfile, new AllocationID());
+ }
+
+
+ /**
+ * Try to allocate a simple slot with specified resource profile and specified allocation id. It's mainly
+ * for testing purpose since we need to specify whatever allocation id we want.
+ */
+ @VisibleForTesting
+ Future<SimpleSlot> allocateSimpleSlot(
+ final JobID jobID,
+ final ResourceProfile resourceProfile,
+ final AllocationID allocationID)
+ {
+ final FlinkCompletableFuture<SlotDescriptor> future = new FlinkCompletableFuture<>();
+
+ internalAllocateSlot(jobID, allocationID, resourceProfile, future);
+
+ final SlotOwner owner = this;
+ return future.thenApplyAsync(
+ new ApplyFunction<SlotDescriptor, SimpleSlot>() {
+ @Override
+ public SimpleSlot apply(SlotDescriptor descriptor) {
+ SimpleSlot slot = new SimpleSlot(
+ descriptor.getJobID(), SlotPool.this,
+ descriptor.getTaskManagerLocation(), descriptor.getSlotNumber(),
+ descriptor.getTaskManagerActorGateway());
+ synchronized (lock) {
+ // double validation since we are out of the lock protection after the slot is granted
+ if (registeredResources.contains(descriptor.getTaskManagerLocation().getResourceID())) {
+ LOG.info("Allocation[{}] Allocated simple slot: {} for job {}.", allocationID, slot, jobID);
+ allocatedSlots.add(allocationID, descriptor, slot);
+ }
+ else {
+ throw new RuntimeException("Resource was marked dead asynchronously.");
+ }
+ }
+ return slot;
+ }
+ },
+ executor
+ );
+ }
+
+
+ /**
+ * Try to allocate a shared slot with specified resource profile.
+ *
+ * @param jobID The job id which the slot allocated for
+ * @param resourceProfile The needed resource profile
+ * @param sharingGroupAssignment The slot sharing group of the vertex
+ * @return The future of allocated shared slot
+ */
+ public Future<SharedSlot> allocateSharedSlot(
+ final JobID jobID,
+ final ResourceProfile resourceProfile,
+ final SlotSharingGroupAssignment sharingGroupAssignment)
+ {
+ return allocateSharedSlot(jobID, resourceProfile, sharingGroupAssignment, new AllocationID());
+ }
+
+ /**
+ * Try to allocate a shared slot with specified resource profile and specified allocation id. It's mainly
+ * for testing purpose since we need to specify whatever allocation id we want.
+ */
+ @VisibleForTesting
+ Future<SharedSlot> allocateSharedSlot(
+ final JobID jobID,
+ final ResourceProfile resourceProfile,
+ final SlotSharingGroupAssignment sharingGroupAssignment,
+ final AllocationID allocationID)
+ {
+ final FlinkCompletableFuture<SlotDescriptor> future = new FlinkCompletableFuture<>();
+
+ internalAllocateSlot(jobID, allocationID, resourceProfile, future);
+
+ return future.thenApplyAsync(
+ new ApplyFunction<SlotDescriptor, SharedSlot>() {
+ @Override
+ public SharedSlot apply(SlotDescriptor descriptor) {
+ SharedSlot slot = new SharedSlot(
+ descriptor.getJobID(), SlotPool.this, descriptor.getTaskManagerLocation(),
+ descriptor.getSlotNumber(), descriptor.getTaskManagerActorGateway(),
+ sharingGroupAssignment);
+
+ synchronized (lock) {
+ // double validation since we are out of the lock protection after the slot is granted
+ if (registeredResources.contains(descriptor.getTaskManagerLocation().getResourceID())) {
+ LOG.info("Allocation[{}] Allocated shared slot: {} for job {}.", allocationID, slot, jobID);
+ allocatedSlots.add(allocationID, descriptor, slot);
+ }
+ else {
+ throw new RuntimeException("Resource was marked dead asynchronously.");
+ }
+ }
+ return slot;
+ }
+ },
+ executor
+ );
+ }
+
+ /**
+ * Internally allocate the slot with specified resource profile. We will first check whether we have some
+ * free slot which can meet the requirement already and allocate it immediately. Otherwise, we will try to
+ * allocation the slot from resource manager.
+ */
+ private void internalAllocateSlot(
+ final JobID jobID,
+ final AllocationID allocationID,
+ final ResourceProfile resourceProfile,
+ final FlinkCompletableFuture<SlotDescriptor> future)
+ {
+ LOG.info("Allocation[{}] Allocating slot with {} for Job {}.", allocationID, resourceProfile, jobID);
+
+ synchronized (lock) {
+ // check whether we have any free slot which can match the required resource profile
+ SlotDescriptor freeSlot = availableSlots.poll(resourceProfile);
+ if (freeSlot != null) {
+ future.complete(freeSlot);
+ }
+ else {
+ if (resourceManagerGateway != null) {
+ LOG.info("Allocation[{}] No available slot exists, trying to allocate from resource manager.",
+ allocationID);
+ SlotRequest slotRequest = new SlotRequest(jobID, allocationID, resourceProfile);
+ pendingRequests.put(allocationID, new Tuple2<>(slotRequest, future));
+ resourceManagerGateway.requestSlot(jobManagerLeaderId, resourceManagerLeaderId, slotRequest, timeout)
+ .handleAsync(new BiFunction<RMSlotRequestReply, Throwable, Void>() {
+ @Override
+ public Void apply(RMSlotRequestReply slotRequestReply, Throwable throwable) {
+ if (throwable != null) {
+ future.completeExceptionally(
+ new Exception("Slot allocation from resource manager failed", throwable));
+ } else if (slotRequestReply instanceof RMSlotRequestRejected) {
+ future.completeExceptionally(
+ new Exception("Slot allocation rejected by resource manager"));
+ }
+ return null;
+ }
+ }, executor);
+ }
+ else {
+ LOG.warn("Allocation[{}] Resource manager not available right now.", allocationID);
+ future.completeExceptionally(new Exception("Resource manager not available right now."));
+ }
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Slot De-allocation
+ // ------------------------------------------------------------------------
+
+ /**
+ * Return the slot back to this pool without releasing it. It's mainly called by failed / cancelled tasks, and the
+ * slot can be reused by other pending requests if the resource profile matches.n
+ *
+ * @param slot The slot needs to be returned
+ * @return True if the returning slot been accepted
+ */
+ @Override
+ public boolean returnAllocatedSlot(Slot slot) {
+ checkNotNull(slot);
+ checkArgument(!slot.isAlive(), "slot is still alive");
+ checkArgument(slot.getOwner() == this, "slot belongs to the wrong pool.");
+
+ if (slot.markReleased()) {
+ synchronized (lock) {
+ final SlotDescriptor slotDescriptor = allocatedSlots.remove(slot);
+ if (slotDescriptor != null) {
+ // check if this TaskManager is valid
+ if (!registeredResources.contains(slot.getTaskManagerID())) {
+ return false;
+ }
+
+ final FlinkCompletableFuture<SlotDescriptor> pendingRequest = pollPendingRequest(slotDescriptor);
+ if (pendingRequest != null) {
+ pendingRequest.complete(slotDescriptor);
+ }
+ else {
+ availableSlots.add(slotDescriptor);
+ }
+
+ return true;
+ }
+ else {
+ throw new IllegalArgumentException("Slot was not allocated from this pool.");
+ }
+ }
+ }
+ else {
+ return false;
+ }
+ }
+
+ private FlinkCompletableFuture<SlotDescriptor> pollPendingRequest(final SlotDescriptor slotDescriptor) {
+ for (Map.Entry<AllocationID, Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>>> entry : pendingRequests.entrySet()) {
+ final Tuple2<SlotRequest, FlinkCompletableFuture<SlotDescriptor>> pendingRequest = entry.getValue();
+ if (slotDescriptor.getResourceProfile().isMatching(pendingRequest.f0.getResourceProfile())) {
+ pendingRequests.remove(entry.getKey());
+ return pendingRequest.f1;
+ }
+ }
+ return null;
+ }
+
+ // ------------------------------------------------------------------------
+ // Slot Releasing
+ // ------------------------------------------------------------------------
+
+ /**
+ * Release slot to TaskManager, called for finished tasks or canceled jobs.
+ *
+ * @param slot The slot needs to be released.
+ */
+ public void releaseSlot(final Slot slot) {
+ synchronized (lock) {
+ allocatedSlots.remove(slot);
+ availableSlots.remove(new SlotDescriptor(slot));
+ // TODO: send release request to task manager
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Slot Offering
+ // ------------------------------------------------------------------------
+
+ /**
+ * Slot offering by TaskManager with AllocationID. The AllocationID is originally generated by this pool and
+ * transfer through the ResourceManager to TaskManager. We use it to distinguish the different allocation
+ * we issued. Slot offering may be rejected if we find something mismatching or there is actually no pending
+ * request waiting for this slot (maybe fulfilled by some other returned slot).
+ *
+ * @param allocationID The allocation id of the lo
+ * @param slotDescriptor The offered slot descriptor
+ * @return True if we accept the offering
+ */
+ public boolean offerSlot(final AllocationID allocationID, final SlotDescriptor slotDescriptor) {
+ synchronized (lock) {
+ // check if this TaskManager is valid
+ final ResourceID resourceID = slotDescriptor.getTaskManagerLocation().getResourceID();
+ if (!registeredResources.contains(resourceID)) {
+ LOG.warn("Allocation[{}] Slot offering from unregistered TaskManager: {}",
+ allocationID, slotDescriptor);
+ return false;
+ }
+
+ // check whether we have already using this slot
+ final Slot allocatedSlot = allocatedSlots.get(allocationID);
+ if (allocatedSlot != null) {
+ final SlotDescriptor allocatedSlotDescriptor = new SlotDescriptor(allocatedSlot);
+
+ if (allocatedSlotDescriptor.equals(slotDescriptor)) {
+ LOG.debug("Allocation[{}] Duplicated slot offering: {}",
+ allocationID, slotDescriptor);
+ return true;
+ }
+ else {
+ LOG.info("Allocation[{}] Allocation had been fulfilled by slot {}, rejecting offered slot {}",
+ allocationID, allocatedSlotDescriptor, slotDescriptor);
+ return false;
+ }
+ }
+
+ // check whether we already have this slot in free pool
+ if (availableSlots.contains(slotDescriptor)) {
+ LOG.debug("Allocation[{}] Duplicated slot offering: {}",
+ allocationID, slotDescriptor);
+ return true;
+ }
+
+ // check whether we have request waiting for this slot
+ if (pendingRequests.containsKey(allocationID)) {
+ FlinkCompletableFuture<SlotDescriptor> future = pendingRequests.remove(allocationID).f1;
+ future.complete(slotDescriptor);
+ return true;
+ }
+
+ // unwanted slot, rejecting this offer
+ return false;
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Resource
+ // ------------------------------------------------------------------------
+
+ /**
+ * Register TaskManager to this pool, only those slots come from registered TaskManager will be considered valid.
+ * Also it provides a way for us to keep "dead" or "abnormal" TaskManagers out of this pool.
+ *
+ * @param resourceID The id of the TaskManager
+ */
+ public void registerResource(final ResourceID resourceID) {
+ synchronized (lock) {
+ registeredResources.add(resourceID);
+ }
+ }
+
+ /**
+ * Unregister TaskManager from this pool, all the related slots will be released and tasks be canceled. Called
+ * when we find some TaskManager becomes "dead" or "abnormal", and we decide to not using slots from it anymore.
+ *
+ * @param resourceID The id of the TaskManager
+ */
+ public void releaseResource(final ResourceID resourceID) {
+ synchronized (lock) {
+ registeredResources.remove(resourceID);
+ availableSlots.removeByResource(resourceID);
+
+ final Set<Slot> allocatedSlotsForResource = allocatedSlots.getSlotsByResource(resourceID);
+ for (Slot slot : allocatedSlotsForResource) {
+ slot.releaseSlot();
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // ResourceManager
+ // ------------------------------------------------------------------------
+
+ public void setResourceManager(
+ final UUID resourceManagerLeaderId,
+ final ResourceManagerGateway resourceManagerGateway)
+ {
+ synchronized (lock) {
+ this.resourceManagerLeaderId = resourceManagerLeaderId;
+ this.resourceManagerGateway = resourceManagerGateway;
+ }
+ }
+
+ public void disconnectResourceManager() {
+ synchronized (lock) {
+ this.resourceManagerLeaderId = null;
+ this.resourceManagerGateway = null;
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Helper classes
+ // ------------------------------------------------------------------------
+
+ /**
+ * Organize allocated slots from different points of view.
+ */
+ static class AllocatedSlots {
+
+ /** All allocated slots organized by TaskManager */
+ private final Map<ResourceID, Set<Slot>> allocatedSlotsByResource;
+
+ /** All allocated slots organized by Slot object */
+ private final Map<Slot, AllocationID> allocatedSlots;
+
+ private final Map<Slot, SlotDescriptor> allocatedSlotsWithDescriptor;
+
+ /** All allocated slots organized by AllocationID */
+ private final Map<AllocationID, Slot> allocatedSlotsById;
+
+ AllocatedSlots() {
+ this.allocatedSlotsByResource = new HashMap<>();
+ this.allocatedSlots = new HashMap<>();
+ this.allocatedSlotsWithDescriptor = new HashMap<>();
+ this.allocatedSlotsById = new HashMap<>();
+ }
+
+ /**
+ * Add a new allocation
+ *
+ * @param allocationID The allocation id
+ * @param slot The allocated slot
+ */
+ void add(final AllocationID allocationID, final SlotDescriptor descriptor, final Slot slot) {
+ allocatedSlots.put(slot, allocationID);
+ allocatedSlotsById.put(allocationID, slot);
+ allocatedSlotsWithDescriptor.put(slot, descriptor);
+
+ final ResourceID resourceID = slot.getTaskManagerID();
+ Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+ if (slotsForResource == null) {
+ slotsForResource = new HashSet<>();
+ allocatedSlotsByResource.put(resourceID, slotsForResource);
+ }
+ slotsForResource.add(slot);
+ }
+
+ /**
+ * Get allocated slot with allocation id
+ *
+ * @param allocationID The allocation id
+ * @return The allocated slot, null if we can't find a match
+ */
+ Slot get(final AllocationID allocationID) {
+ return allocatedSlotsById.get(allocationID);
+ }
+
+ /**
+ * Check whether we have allocated this slot
+ *
+ * @param slot The slot needs to checked
+ * @return True if we contains this slot
+ */
+ boolean contains(final Slot slot) {
+ return allocatedSlots.containsKey(slot);
+ }
+
+ /**
+ * Remove an allocation with slot.
+ *
+ * @param slot The slot needs to be removed
+ */
+ SlotDescriptor remove(final Slot slot) {
+ final SlotDescriptor descriptor = allocatedSlotsWithDescriptor.remove(slot);
+ if (descriptor != null) {
+ final AllocationID allocationID = allocatedSlots.remove(slot);
+ if (allocationID != null) {
+ allocatedSlotsById.remove(allocationID);
+ } else {
+ throw new IllegalStateException("Bug: maps are inconsistent");
+ }
+
+ final ResourceID resourceID = slot.getTaskManagerID();
+ final Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+ slotsForResource.remove(slot);
+ if (slotsForResource.isEmpty()) {
+ allocatedSlotsByResource.remove(resourceID);
+ }
+
+ return descriptor;
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Get all allocated slot from same TaskManager.
+ *
+ * @param resourceID The id of the TaskManager
+ * @return Set of slots which are allocated from the same TaskManager
+ */
+ Set<Slot> getSlotsByResource(final ResourceID resourceID) {
+ Set<Slot> slotsForResource = allocatedSlotsByResource.get(resourceID);
+ if (slotsForResource != null) {
+ return new HashSet<>(slotsForResource);
+ }
+ else {
+ return new HashSet<>();
+ }
+ }
+
+ @VisibleForTesting
+ boolean containResource(final ResourceID resourceID) {
+ return allocatedSlotsByResource.containsKey(resourceID);
+ }
+
+ @VisibleForTesting
+ int size() {
+ return allocatedSlots.size();
+ }
+ }
+
+ /**
+ * Organize all available slots from different points of view.
+ */
+ static class AvailableSlots {
+
+ /** All available slots organized by TaskManager */
+ private final Map<ResourceID, Set<SlotDescriptor>> availableSlotsByResource;
+
+ /** All available slots */
+ private final Set<SlotDescriptor> availableSlots;
+
+ AvailableSlots() {
+ this.availableSlotsByResource = new HashMap<>();
+ this.availableSlots = new HashSet<>();
+ }
+
+ /**
+ * Add an available slot.
+ *
+ * @param descriptor The descriptor of the slot
+ */
+ void add(final SlotDescriptor descriptor) {
+ availableSlots.add(descriptor);
+
+ final ResourceID resourceID = descriptor.getTaskManagerLocation().getResourceID();
+ Set<SlotDescriptor> slotsForResource = availableSlotsByResource.get(resourceID);
+ if (slotsForResource == null) {
+ slotsForResource = new HashSet<>();
+ availableSlotsByResource.put(resourceID, slotsForResource);
+ }
+ slotsForResource.add(descriptor);
+ }
+
+ /**
+ * Check whether we have this slot
+ *
+ * @param slotDescriptor The descriptor of the slot
+ * @return True if we contains this slot
+ */
+ boolean contains(final SlotDescriptor slotDescriptor) {
+ return availableSlots.contains(slotDescriptor);
+ }
+
+ /**
+ * Poll a slot which matches the required resource profile
+ *
+ * @param resourceProfile The required resource profile
+ * @return Slot which matches the resource profile, null if we can't find a match
+ */
+ SlotDescriptor poll(final ResourceProfile resourceProfile) {
+ for (SlotDescriptor slotDescriptor : availableSlots) {
+ if (slotDescriptor.getResourceProfile().isMatching(resourceProfile)) {
+ remove(slotDescriptor);
+ return slotDescriptor;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Remove all available slots come from specified TaskManager.
+ *
+ * @param resourceID The id of the TaskManager
+ */
+ void removeByResource(final ResourceID resourceID) {
+ final Set<SlotDescriptor> slotsForResource = availableSlotsByResource.remove(resourceID);
+ if (slotsForResource != null) {
+ for (SlotDescriptor slotDescriptor : slotsForResource) {
+ availableSlots.remove(slotDescriptor);
+ }
+ }
+ }
+
+ private void remove(final SlotDescriptor slotDescriptor) {
+ availableSlots.remove(slotDescriptor);
+
+ final ResourceID resourceID = slotDescriptor.getTaskManagerLocation().getResourceID();
+ final Set<SlotDescriptor> slotsForResource = checkNotNull(availableSlotsByResource.get(resourceID));
+ slotsForResource.remove(slotDescriptor);
+ if (slotsForResource.isEmpty()) {
+ availableSlotsByResource.remove(resourceID);
+ }
+ }
+
+ @VisibleForTesting
+ boolean containResource(final ResourceID resourceID) {
+ return availableSlotsByResource.containsKey(resourceID);
+ }
+
+ @VisibleForTesting
+ int size() {
+ return availableSlots.size();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
index 9209d15..2461340 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/akka/AkkaUtils.scala
@@ -583,8 +583,8 @@ object AkkaUtils {
}
def formatDurationParingErrorMessage: String = {
- "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " +
- "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|"+
+ "Duration format must be \"val unit\", where 'val' is a number and 'unit' is " +
+ "(d|day)|(h|hour)|(min|minute)|s|sec|second)|(ms|milli|millisecond)|" +
"(�s|micro|microsecond)|(ns|nano|nanosecond)"
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
new file mode 100644
index 0000000..655a3ea
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotsTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AllocatedSlotsTest {
+
+ @Test
+ public void testOperations() throws Exception {
+ SlotPool.AllocatedSlots allocatedSlots = new SlotPool.AllocatedSlots();
+
+ final AllocationID allocation1 = new AllocationID();
+ final ResourceID resource1 = new ResourceID("resource1");
+ final Slot slot1 = createSlot(resource1);
+
+ allocatedSlots.add(allocation1, new SlotDescriptor(slot1), slot1);
+
+ assertTrue(allocatedSlots.contains(slot1));
+ assertTrue(allocatedSlots.containResource(resource1));
+
+ assertEquals(slot1, allocatedSlots.get(allocation1));
+ assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(1, allocatedSlots.size());
+
+ final AllocationID allocation2 = new AllocationID();
+ final Slot slot2 = createSlot(resource1);
+
+ allocatedSlots.add(allocation2, new SlotDescriptor(slot2), slot2);
+
+ assertTrue(allocatedSlots.contains(slot1));
+ assertTrue(allocatedSlots.contains(slot2));
+ assertTrue(allocatedSlots.containResource(resource1));
+
+ assertEquals(slot1, allocatedSlots.get(allocation1));
+ assertEquals(slot2, allocatedSlots.get(allocation2));
+ assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(2, allocatedSlots.size());
+
+ final AllocationID allocation3 = new AllocationID();
+ final ResourceID resource2 = new ResourceID("resource2");
+ final Slot slot3 = createSlot(resource2);
+
+ allocatedSlots.add(allocation3, new SlotDescriptor(slot2), slot3);
+
+ assertTrue(allocatedSlots.contains(slot1));
+ assertTrue(allocatedSlots.contains(slot2));
+ assertTrue(allocatedSlots.contains(slot3));
+ assertTrue(allocatedSlots.containResource(resource1));
+ assertTrue(allocatedSlots.containResource(resource2));
+
+ assertEquals(slot1, allocatedSlots.get(allocation1));
+ assertEquals(slot2, allocatedSlots.get(allocation2));
+ assertEquals(slot3, allocatedSlots.get(allocation3));
+ assertEquals(2, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+ assertEquals(3, allocatedSlots.size());
+
+ allocatedSlots.remove(slot2);
+
+ assertTrue(allocatedSlots.contains(slot1));
+ assertFalse(allocatedSlots.contains(slot2));
+ assertTrue(allocatedSlots.contains(slot3));
+ assertTrue(allocatedSlots.containResource(resource1));
+ assertTrue(allocatedSlots.containResource(resource2));
+
+ assertEquals(slot1, allocatedSlots.get(allocation1));
+ assertNull(allocatedSlots.get(allocation2));
+ assertEquals(slot3, allocatedSlots.get(allocation3));
+ assertEquals(1, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+ assertEquals(2, allocatedSlots.size());
+
+ allocatedSlots.remove(slot1);
+
+ assertFalse(allocatedSlots.contains(slot1));
+ assertFalse(allocatedSlots.contains(slot2));
+ assertTrue(allocatedSlots.contains(slot3));
+ assertFalse(allocatedSlots.containResource(resource1));
+ assertTrue(allocatedSlots.containResource(resource2));
+
+ assertNull(allocatedSlots.get(allocation1));
+ assertNull(allocatedSlots.get(allocation2));
+ assertEquals(slot3, allocatedSlots.get(allocation3));
+ assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(1, allocatedSlots.getSlotsByResource(resource2).size());
+ assertEquals(1, allocatedSlots.size());
+
+ allocatedSlots.remove(slot3);
+
+ assertFalse(allocatedSlots.contains(slot1));
+ assertFalse(allocatedSlots.contains(slot2));
+ assertFalse(allocatedSlots.contains(slot3));
+ assertFalse(allocatedSlots.containResource(resource1));
+ assertFalse(allocatedSlots.containResource(resource2));
+
+ assertNull(allocatedSlots.get(allocation1));
+ assertNull(allocatedSlots.get(allocation2));
+ assertNull(allocatedSlots.get(allocation3));
+ assertEquals(0, allocatedSlots.getSlotsByResource(resource1).size());
+ assertEquals(0, allocatedSlots.getSlotsByResource(resource2).size());
+ assertEquals(0, allocatedSlots.size());
+ }
+
+ private Slot createSlot(final ResourceID resourceId) {
+ Slot slot = mock(Slot.class);
+ when(slot.getTaskManagerID()).thenReturn(resourceId);
+ return slot;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
new file mode 100644
index 0000000..872810f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AvailableSlotsTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class AvailableSlotsTest {
+
+ static final ResourceProfile DEFAULT_TESTING_PROFILE = new ResourceProfile(1.0, 512);
+
+ static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = new ResourceProfile(2.0, 1024);
+
+ @Test
+ public void testAddAndRemove() throws Exception {
+ SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+ final ResourceID resource1 = new ResourceID("resource1");
+ final ResourceID resource2 = new ResourceID("resource2");
+
+ final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+ final SlotDescriptor slot2 = createSlotDescriptor(resource1);
+ final SlotDescriptor slot3 = createSlotDescriptor(resource2);
+
+ availableSlots.add(slot1);
+ availableSlots.add(slot2);
+ availableSlots.add(slot3);
+
+ assertEquals(3, availableSlots.size());
+ assertTrue(availableSlots.contains(slot1));
+ assertTrue(availableSlots.contains(slot2));
+ assertTrue(availableSlots.contains(slot3));
+ assertTrue(availableSlots.containResource(resource1));
+ assertTrue(availableSlots.containResource(resource2));
+
+ availableSlots.removeByResource(resource1);
+
+ assertEquals(1, availableSlots.size());
+ assertFalse(availableSlots.contains(slot1));
+ assertFalse(availableSlots.contains(slot2));
+ assertTrue(availableSlots.contains(slot3));
+ assertFalse(availableSlots.containResource(resource1));
+ assertTrue(availableSlots.containResource(resource2));
+
+ availableSlots.removeByResource(resource2);
+
+ assertEquals(0, availableSlots.size());
+ assertFalse(availableSlots.contains(slot1));
+ assertFalse(availableSlots.contains(slot2));
+ assertFalse(availableSlots.contains(slot3));
+ assertFalse(availableSlots.containResource(resource1));
+ assertFalse(availableSlots.containResource(resource2));
+ }
+
+ @Test
+ public void testPollFreeSlot() {
+ SlotPool.AvailableSlots availableSlots = new SlotPool.AvailableSlots();
+
+ final ResourceID resource1 = new ResourceID("resource1");
+ final SlotDescriptor slot1 = createSlotDescriptor(resource1);
+
+ availableSlots.add(slot1);
+
+ assertEquals(1, availableSlots.size());
+ assertTrue(availableSlots.contains(slot1));
+ assertTrue(availableSlots.containResource(resource1));
+
+ assertNull(availableSlots.poll(DEFAULT_TESTING_BIG_PROFILE));
+
+ assertEquals(slot1, availableSlots.poll(DEFAULT_TESTING_PROFILE));
+ assertEquals(0, availableSlots.size());
+ assertFalse(availableSlots.contains(slot1));
+ assertFalse(availableSlots.containResource(resource1));
+ }
+
+ static SlotDescriptor createSlotDescriptor(final ResourceID resourceID) {
+ return createSlotDescriptor(resourceID, new JobID());
+ }
+
+ static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID) {
+ return createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ }
+
+ static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+ final ResourceProfile resourceProfile)
+ {
+ return createSlotDescriptor(resourceID, jobID, resourceProfile, 0);
+ }
+
+ static SlotDescriptor createSlotDescriptor(final ResourceID resourceID, final JobID jobID,
+ final ResourceProfile resourceProfile, final int slotNumber)
+ {
+ TaskManagerLocation location = mock(TaskManagerLocation.class);
+ when(location.getResourceID()).thenReturn(resourceID);
+ return new SlotDescriptor(jobID, location, slotNumber, resourceProfile, mock(ActorGateway.class));
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/48c936ee/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java
new file mode 100644
index 0000000..30cdbd6
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.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.instance;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.concurrent.BiFunction;
+import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.resourcemanager.SlotRequest;
+import org.apache.flink.util.TestLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.DEFAULT_TESTING_PROFILE;
+import static org.apache.flink.runtime.instance.AvailableSlotsTest.createSlotDescriptor;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class SlotPoolTest extends TestLogger {
+
+ private Executor executor;
+
+ private SlotPool slotPool;
+
+ private ResourceManagerGateway resourceManagerGateway;
+
+ @Before
+ public void setUp() throws Exception {
+ this.executor = Executors.newFixedThreadPool(1);
+ this.slotPool = new SlotPool(executor);
+ this.resourceManagerGateway = mock(ResourceManagerGateway.class);
+ when(resourceManagerGateway
+ .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class)))
+ .thenReturn(mock(Future.class));
+ slotPool.setResourceManager(UUID.randomUUID(), resourceManagerGateway);
+ slotPool.setJobManagerLeaderId(UUID.randomUUID());
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ }
+
+ @Test
+ public void testAllocateSimpleSlot() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobID jobID = new JobID();
+ AllocationID allocationID = new AllocationID();
+ Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+ assertFalse(future.isDone());
+ verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+ SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+ assertTrue(future.isDone());
+ assertTrue(slot.isAlive());
+ assertEquals(resourceID, slot.getTaskManagerID());
+ assertEquals(jobID, slot.getJobID());
+ assertEquals(slotPool, slot.getOwner());
+ }
+
+ @Test
+ public void testAllocateSharedSlot() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobVertexID vid = new JobVertexID();
+ SlotSharingGroup sharingGroup = new SlotSharingGroup(vid);
+ SlotSharingGroupAssignment assignment = sharingGroup.getTaskAssignment();
+
+ JobID jobID = new JobID();
+ AllocationID allocationID = new AllocationID();
+ Future<SharedSlot> future = slotPool.allocateSharedSlot(jobID, DEFAULT_TESTING_PROFILE, assignment, allocationID);
+
+ assertFalse(future.isDone());
+ verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+
+ SharedSlot slot = future.get(1, TimeUnit.SECONDS);
+ assertTrue(future.isDone());
+ assertTrue(slot.isAlive());
+ assertEquals(resourceID, slot.getTaskManagerID());
+ assertEquals(jobID, slot.getJobID());
+ assertEquals(slotPool, slot.getOwner());
+
+ SimpleSlot simpleSlot = slot.allocateSubSlot(vid);
+ assertNotNull(simpleSlot);
+ assertTrue(simpleSlot.isAlive());
+ }
+
+ @Test
+ public void testAllocateSlotWithoutResourceManager() throws Exception {
+ slotPool.disconnectResourceManager();
+ Future<SimpleSlot> future = slotPool.allocateSimpleSlot(new JobID(), DEFAULT_TESTING_PROFILE);
+ future.handleAsync(
+ new BiFunction<SimpleSlot, Throwable, Void>() {
+ @Override
+ public Void apply(SimpleSlot simpleSlot, Throwable throwable) {
+ assertNull(simpleSlot);
+ assertNotNull(throwable);
+ return null;
+ }
+ },
+ executor);
+ try {
+ future.get(1, TimeUnit.SECONDS);
+ fail("We expected a ExecutionException.");
+ } catch (ExecutionException ex) {
+ // we expect the exception
+ }
+ }
+
+ @Test
+ public void testAllocationFulfilledByReturnedSlot() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobID jobID = new JobID();
+
+ AllocationID allocationID1 = new AllocationID();
+ Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+ AllocationID allocationID2 = new AllocationID();
+ Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+ assertFalse(future1.isDone());
+ assertFalse(future2.isDone());
+ verify(resourceManagerGateway, times(2))
+ .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+ SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+ assertTrue(future1.isDone());
+ assertFalse(future2.isDone());
+
+ // return this slot to pool
+ slot1.releaseSlot();
+
+ // second allocation fulfilled by previous slot returning
+ SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+ assertTrue(future2.isDone());
+
+ assertNotEquals(slot1, slot2);
+ assertTrue(slot1.isReleased());
+ assertTrue(slot2.isAlive());
+ assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+ assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+ }
+
+ @Test
+ public void testAllocateWithFreeSlot() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobID jobID = new JobID();
+ AllocationID allocationID1 = new AllocationID();
+ Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+ assertFalse(future1.isDone());
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+ SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+ assertTrue(future1.isDone());
+
+ // return this slot to pool
+ slot1.releaseSlot();
+
+ AllocationID allocationID2 = new AllocationID();
+ Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+ // second allocation fulfilled by previous slot returning
+ SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS);
+ assertTrue(future2.isDone());
+
+ assertNotEquals(slot1, slot2);
+ assertTrue(slot1.isReleased());
+ assertTrue(slot2.isAlive());
+ assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID());
+ assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber());
+ }
+
+ @Test
+ public void testOfferSlot() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobID jobID = new JobID();
+ AllocationID allocationID = new AllocationID();
+ Future<SimpleSlot> future = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID);
+ assertFalse(future.isDone());
+ verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class));
+
+ // slot from unregistered resource
+ SlotDescriptor invalid = createSlotDescriptor(new ResourceID("unregistered"), jobID, DEFAULT_TESTING_PROFILE);
+ assertFalse(slotPool.offerSlot(allocationID, invalid));
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+
+ // reject offering with mismatch allocation id
+ assertFalse(slotPool.offerSlot(new AllocationID(), slotDescriptor));
+
+ // accepted slot
+ assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+ SimpleSlot slot = future.get(1, TimeUnit.SECONDS);
+ assertTrue(future.isDone());
+ assertTrue(slot.isAlive());
+
+ // conflict offer with using slot
+ SlotDescriptor conflict = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertFalse(slotPool.offerSlot(allocationID, conflict));
+
+ // duplicated offer with using slot
+ assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+ assertTrue(future.isDone());
+ assertTrue(slot.isAlive());
+
+ // duplicated offer with free slot
+ slot.releaseSlot();
+ assertTrue(slot.isReleased());
+ assertTrue(slotPool.offerSlot(allocationID, slotDescriptor));
+ }
+
+ @Test
+ public void testReleaseResource() throws Exception {
+ ResourceID resourceID = new ResourceID("resource");
+ slotPool.registerResource(resourceID);
+
+ JobID jobID = new JobID();
+
+ AllocationID allocationID1 = new AllocationID();
+ Future<SimpleSlot> future1 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID1);
+
+ AllocationID allocationID2 = new AllocationID();
+ Future<SimpleSlot> future2 = slotPool.allocateSimpleSlot(jobID, DEFAULT_TESTING_PROFILE, allocationID2);
+
+ SlotDescriptor slotDescriptor = createSlotDescriptor(resourceID, jobID, DEFAULT_TESTING_PROFILE);
+ assertTrue(slotPool.offerSlot(allocationID1, slotDescriptor));
+
+ SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS);
+ assertTrue(future1.isDone());
+ assertFalse(future2.isDone());
+
+ slotPool.releaseResource(resourceID);
+ assertTrue(slot1.isReleased());
+
+ // slot released and not usable, second allocation still not fulfilled
+ Thread.sleep(10);
+ assertFalse(future2.isDone());
+ }
+
+}
[2/8] flink git commit: [FLINK-4375] [distributed coordination]
Implement new JobManager creation, initialization, and basic RPC methods
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index 3b8fc97..d9ff88f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -18,38 +18,31 @@
package org.apache.flink.runtime.jobmaster;
-import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.time.Time;
-import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
-import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
-import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore;
-import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker;
-import org.apache.flink.runtime.checkpoint.stats.SimpleCheckpointStatsTracker;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.client.JobSubmissionException;
import org.apache.flink.runtime.client.SerializedJobExecutionResult;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.concurrent.BiFunction;
import org.apache.flink.runtime.concurrent.Future;
-import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder;
import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.JobStatusListener;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
@@ -59,18 +52,13 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
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.JobVertexID;
-import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
-import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
@@ -83,9 +71,12 @@ import org.apache.flink.runtime.registration.RegisteredRpcConnection;
import org.apache.flink.runtime.registration.RegistrationResponse;
import org.apache.flink.runtime.registration.RetryingRegistration;
import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcMethod;
import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.state.CheckpointStateHandles;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import org.apache.flink.runtime.util.SerializedThrowable;
@@ -93,15 +84,13 @@ import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.SerializedValue;
import org.slf4j.Logger;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.duration.FiniteDuration;
-
+import javax.annotation.Nullable;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -112,16 +101,21 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* It offers the following methods as part of its rpc interface to interact with the JobMaster
* remotely:
* <ul>
- * <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
+ * <li>{@link #updateTaskExecutionState} updates the task execution state for
* given task</li>
* </ul>
*/
public class JobMaster extends RpcEndpoint<JobMasterGateway> {
+ private static final AtomicReferenceFieldUpdater<JobMaster, UUID> LEADER_ID_UPDATER =
+ AtomicReferenceFieldUpdater.newUpdater(JobMaster.class, UUID.class, "leaderSessionID");
+
+ // ------------------------------------------------------------------------
+
/** Logical representation of the job */
private final JobGraph jobGraph;
- /** Configuration of the job */
+ /** Configuration of the JobManager */
private final Configuration configuration;
/** Service to contend for and retrieve the leadership of JM and RM */
@@ -130,37 +124,24 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
/** Blob cache manager used across jobs */
private final BlobLibraryCacheManager libraryCacheManager;
- /** Factory to create restart strategy for this job */
- private final RestartStrategyFactory restartStrategyFactory;
-
- /** Store for save points */
- private final SavepointStore savepointStore;
-
- /** The timeout for this job */
- private final Time timeout;
-
- /** The scheduler to use for scheduling new tasks as they are needed */
- private final Scheduler scheduler;
+ /** The metrics for the JobManager itself */
+ private final MetricGroup jobManagerMetricGroup;
- /** The metrics group used across jobs */
- private final JobManagerMetricGroup jobManagerMetricGroup;
+ /** The metrics for the job */
+ private final MetricGroup jobMetricGroup;
/** The execution context which is used to execute futures */
- private final Executor executionContext;
+ private final ExecutorService executionContext;
private final OnCompletionActions jobCompletionActions;
- /** The execution graph of this job */
- private volatile ExecutionGraph executionGraph;
-
- /** The checkpoint recovery factory used by this job */
- private CheckpointRecoveryFactory checkpointRecoveryFactory;
+ private final FatalErrorHandler errorHandler;
- private ClassLoader userCodeLoader;
+ private final ClassLoader userCodeLoader;
- private RestartStrategy restartStrategy;
+ /** The execution graph of this job */
+ private final ExecutionGraph executionGraph;
- private MetricGroup jobMetrics;
private volatile UUID leaderSessionID;
@@ -170,22 +151,27 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
private LeaderRetrievalService resourceManagerLeaderRetriever;
/** Connection with ResourceManager, null if not located address yet or we close it initiative */
- private volatile ResourceManagerConnection resourceManagerConnection;
+ private ResourceManagerConnection resourceManagerConnection;
+
+ // TODO - we need to replace this with the slot pool
+ private final Scheduler scheduler;
// ------------------------------------------------------------------------
public JobMaster(
- JobGraph jobGraph,
- Configuration configuration,
- RpcService rpcService,
- HighAvailabilityServices highAvailabilityService,
- BlobLibraryCacheManager libraryCacheManager,
- RestartStrategyFactory restartStrategyFactory,
- SavepointStore savepointStore,
- Time timeout,
- Scheduler scheduler,
- JobManagerMetricGroup jobManagerMetricGroup,
- OnCompletionActions jobCompletionActions)
+ JobGraph jobGraph,
+ Configuration configuration,
+ RpcService rpcService,
+ HighAvailabilityServices highAvailabilityService,
+ ExecutorService executorService,
+ BlobLibraryCacheManager libraryCacheManager,
+ RestartStrategyFactory restartStrategyFactory,
+ SavepointStore savepointStore,
+ Time rpcAskTimeout,
+ @Nullable JobManagerMetricGroup jobManagerMetricGroup,
+ OnCompletionActions jobCompletionActions,
+ FatalErrorHandler errorHandler,
+ ClassLoader userCodeLoader) throws Exception
{
super(rpcService);
@@ -193,289 +179,150 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
this.configuration = checkNotNull(configuration);
this.highAvailabilityServices = checkNotNull(highAvailabilityService);
this.libraryCacheManager = checkNotNull(libraryCacheManager);
- this.restartStrategyFactory = checkNotNull(restartStrategyFactory);
- this.savepointStore = checkNotNull(savepointStore);
- this.timeout = checkNotNull(timeout);
- this.scheduler = checkNotNull(scheduler);
- this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup);
- this.executionContext = checkNotNull(rpcService.getExecutor());
+ this.executionContext = checkNotNull(executorService);
this.jobCompletionActions = checkNotNull(jobCompletionActions);
- }
+ this.errorHandler = checkNotNull(errorHandler);
+ this.userCodeLoader = checkNotNull(userCodeLoader);
- //----------------------------------------------------------------------------------------------
- // Lifecycle management
- //----------------------------------------------------------------------------------------------
-
- /**
- * Initializing the job execution environment, should be called before start. Any error occurred during
- * initialization will be treated as job submission failure.
- *
- * @throws JobSubmissionException
- */
- public void init() throws JobSubmissionException {
- log.info("Initializing job {} ({}).", jobGraph.getName(), jobGraph.getJobID());
-
- try {
- // IMPORTANT: We need to make sure that the library registration is the first action,
- // because this makes sure that the uploaded jar files are removed in case of
- // unsuccessful
- try {
- libraryCacheManager.registerJob(jobGraph.getJobID(), jobGraph.getUserJarBlobKeys(),
- jobGraph.getClasspaths());
- } catch (Throwable t) {
- throw new JobSubmissionException(jobGraph.getJobID(),
- "Cannot set up the user code libraries: " + t.getMessage(), t);
- }
+ final String jobName = jobGraph.getName();
+ final JobID jid = jobGraph.getJobID();
- userCodeLoader = libraryCacheManager.getClassLoader(jobGraph.getJobID());
- if (userCodeLoader == null) {
- throw new JobSubmissionException(jobGraph.getJobID(),
- "The user code class loader could not be initialized.");
- }
+ if (jobManagerMetricGroup != null) {
+ this.jobManagerMetricGroup = jobManagerMetricGroup;
+ this.jobMetricGroup = jobManagerMetricGroup.addJob(jobGraph);
+ } else {
+ this.jobManagerMetricGroup = new UnregisteredMetricsGroup();
+ this.jobMetricGroup = new UnregisteredMetricsGroup();
+ }
- if (jobGraph.getNumberOfVertices() == 0) {
- throw new JobSubmissionException(jobGraph.getJobID(), "The given job is empty");
- }
+ log.info("Initializing job {} ({}).", jobName, jid);
- final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
+ final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration =
jobGraph.getSerializedExecutionConfig()
- .deserializeValue(userCodeLoader)
- .getRestartStrategy();
- if (restartStrategyConfiguration != null) {
- restartStrategy = RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration);
- }
- else {
- restartStrategy = restartStrategyFactory.createRestartStrategy();
- }
+ .deserializeValue(userCodeLoader)
+ .getRestartStrategy();
- log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobGraph.getName(), jobGraph.getJobID());
+ final RestartStrategy restartStrategy = (restartStrategyConfiguration != null) ?
+ RestartStrategyFactory.createRestartStrategy(restartStrategyConfiguration) :
+ restartStrategyFactory.createRestartStrategy();
- if (jobManagerMetricGroup != null) {
- jobMetrics = jobManagerMetricGroup.addJob(jobGraph);
- }
- if (jobMetrics == null) {
- jobMetrics = new UnregisteredMetricsGroup();
- }
+ log.info("Using restart strategy {} for {} ({}).", restartStrategy, jobName, jid);
- try {
- checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
- } catch (Exception e) {
- log.error("Could not get the checkpoint recovery factory.", e);
- throw new JobSubmissionException(jobGraph.getJobID(), "Could not get the checkpoint recovery factory.", e);
- }
+ CheckpointRecoveryFactory checkpointRecoveryFactory;
+ try {
+ checkpointRecoveryFactory = highAvailabilityServices.getCheckpointRecoveryFactory();
+ } catch (Exception e) {
+ log.error("Could not create the access to highly-available checkpoint storage.", e);
+ throw new Exception("Could not create the access to highly-available checkpoint storage.", e);
+ }
- try {
- resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
- } catch (Exception e) {
- log.error("Could not get the resource manager leader retriever.", e);
- throw new JobSubmissionException(jobGraph.getJobID(),
+ try {
+ resourceManagerLeaderRetriever = highAvailabilityServices.getResourceManagerLeaderRetriever();
+ } catch (Exception e) {
+ log.error("Could not get the resource manager leader retriever.", e);
+ throw new JobSubmissionException(jobGraph.getJobID(),
"Could not get the resource manager leader retriever.", e);
- }
- } catch (Throwable t) {
- log.error("Failed to initializing job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
+ }
- libraryCacheManager.unregisterJob(jobGraph.getJobID());
+ this.executionGraph = ExecutionGraphBuilder.buildGraph(
+ null,
+ jobGraph,
+ configuration,
+ executorService,
+ userCodeLoader,
+ checkpointRecoveryFactory,
+ savepointStore,
+ rpcAskTimeout,
+ restartStrategy,
+ jobMetricGroup,
+ -1,
+ log);
- if (t instanceof JobSubmissionException) {
- throw (JobSubmissionException) t;
- }
- else {
- throw new JobSubmissionException(jobGraph.getJobID(), "Failed to initialize job " +
- jobGraph.getName() + " (" + jobGraph.getJobID() + ")", t);
- }
- }
+ // TODO - temp fix
+ this.scheduler = new Scheduler(executorService);
}
+ //----------------------------------------------------------------------------------------------
+ // Lifecycle management
+ //----------------------------------------------------------------------------------------------
+
+
@Override
public void start() {
- super.start();
+ throw new UnsupportedOperationException("Should never call start() without leader ID");
}
+ /**
+ * Start the rpc service and begin to run the job.
+ *
+ * @param leaderSessionID The necessary leader id for running the job.
+ */
+ public void start(final UUID leaderSessionID) {
+ if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) {
+ super.start();
+
+ log.info("Starting JobManager for job {} ({})", jobGraph.getName(), jobGraph.getJobID());
+ getSelf().startJobExecution();
+ } else {
+ log.warn("Job already started with leaderId {}, ignoring this start request.", leaderSessionID);
+ }
+ }
+
+ /**
+ * Suspend the job and shutdown all other services including rpc.
+ */
@Override
public void shutDown() {
+ // make sure there is a graceful exit
+ getSelf().suspendExecution(new Exception("JobManager is shutting down."));
super.shutDown();
-
- suspendJob(new Exception("JobManager is shutting down."));
-
- disposeCommunicationWithResourceManager();
}
-
-
//----------------------------------------------------------------------------------------------
// RPC methods
//----------------------------------------------------------------------------------------------
- /**
- * Start to run the job, runtime data structures like ExecutionGraph will be constructed now and checkpoint
- * being recovered. After this, we will begin to schedule the job.
- */
+ //-- job starting and stopping -----------------------------------------------------------------
+
@RpcMethod
- public void startJob(final UUID leaderSessionID) {
- log.info("Starting job {} ({}) with leaderId {}.", jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
-
- this.leaderSessionID = leaderSessionID;
-
- if (executionGraph != null) {
- executionGraph = new ExecutionGraph(
- ExecutionContext$.MODULE$.fromExecutor(executionContext),
- jobGraph.getJobID(),
- jobGraph.getName(),
- jobGraph.getJobConfiguration(),
- jobGraph.getSerializedExecutionConfig(),
- new FiniteDuration(timeout.getSize(), timeout.getUnit()),
- restartStrategy,
- jobGraph.getUserJarBlobKeys(),
- jobGraph.getClasspaths(),
- userCodeLoader,
- jobMetrics);
- }
- else {
- // TODO: update last active time in JobInfo
- }
+ public void startJobExecution() {
+ log.info("Starting execution of job {} ({}) with leaderId {}.",
+ jobGraph.getName(), jobGraph.getJobID(), leaderSessionID);
try {
- executionGraph.setScheduleMode(jobGraph.getScheduleMode());
- executionGraph.setQueuedSchedulingAllowed(jobGraph.getAllowQueuedScheduling());
-
- try {
- executionGraph.setJsonPlan(JsonPlanGenerator.generatePlan(jobGraph));
- } catch (Exception e) {
- log.warn("Cannot create JSON plan for job {} ({})", jobGraph.getJobID(), jobGraph.getName(), e);
- executionGraph.setJsonPlan("{}");
- }
-
- // initialize the vertices that have a master initialization hook
- // file output formats create directories here, input formats create splits
- if (log.isDebugEnabled()) {
- log.debug("Running initialization on master for job {} ({}).", jobGraph.getJobID(), jobGraph.getName());
- }
- for (JobVertex vertex : jobGraph.getVertices()) {
- final String executableClass = vertex.getInvokableClassName();
- if (executableClass == null || executableClass.length() == 0) {
- throw new JobExecutionException(jobGraph.getJobID(),
- "The vertex " + vertex.getID() + " (" + vertex.getName() + ") has no invokable class.");
- }
- if (vertex.getParallelism() == ExecutionConfig.PARALLELISM_AUTO_MAX) {
- vertex.setParallelism(scheduler.getTotalNumberOfSlots());
- }
-
- try {
- vertex.initializeOnMaster(userCodeLoader);
- } catch (Throwable t) {
- throw new JobExecutionException(jobGraph.getJobID(),
- "Cannot initialize task '" + vertex.getName() + "': " + t.getMessage(), t);
- }
- }
-
- // topologically sort the job vertices and attach the graph to the existing one
- final List<JobVertex> sortedTopology = jobGraph.getVerticesSortedTopologicallyFromSources();
- if (log.isDebugEnabled()) {
- log.debug("Adding {} vertices from job graph {} ({}).", sortedTopology.size(),
- jobGraph.getJobID(), jobGraph.getName());
- }
- executionGraph.attachJobGraph(sortedTopology);
-
- if (log.isDebugEnabled()) {
- log.debug("Successfully created execution graph from job graph {} ({}).",
- jobGraph.getJobID(), jobGraph.getName());
- }
-
- final JobSnapshottingSettings snapshotSettings = jobGraph.getSnapshotSettings();
- if (snapshotSettings != null) {
- List<ExecutionJobVertex> triggerVertices = getExecutionJobVertexWithId(
- executionGraph, snapshotSettings.getVerticesToTrigger());
-
- List<ExecutionJobVertex> ackVertices = getExecutionJobVertexWithId(
- executionGraph, snapshotSettings.getVerticesToAcknowledge());
-
- List<ExecutionJobVertex> confirmVertices = getExecutionJobVertexWithId(
- executionGraph, snapshotSettings.getVerticesToConfirm());
-
- CompletedCheckpointStore completedCheckpoints = checkpointRecoveryFactory.createCheckpointStore(
- jobGraph.getJobID(), userCodeLoader);
-
- CheckpointIDCounter checkpointIdCounter = checkpointRecoveryFactory.createCheckpointIDCounter(
- jobGraph.getJobID());
-
- // Checkpoint stats tracker
- boolean isStatsDisabled = configuration.getBoolean(
- ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_DISABLE,
- ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_DISABLE);
-
- final CheckpointStatsTracker checkpointStatsTracker;
- if (isStatsDisabled) {
- checkpointStatsTracker = new DisabledCheckpointStatsTracker();
- }
- else {
- int historySize = configuration.getInteger(
- ConfigConstants.JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE,
- ConfigConstants.DEFAULT_JOB_MANAGER_WEB_CHECKPOINTS_HISTORY_SIZE);
- checkpointStatsTracker = new SimpleCheckpointStatsTracker(historySize, ackVertices, jobMetrics);
- }
-
- executionGraph.enableSnapshotCheckpointing(
- snapshotSettings.getCheckpointInterval(),
- snapshotSettings.getCheckpointTimeout(),
- snapshotSettings.getMinPauseBetweenCheckpoints(),
- snapshotSettings.getMaxConcurrentCheckpoints(),
- triggerVertices,
- ackVertices,
- confirmVertices,
- checkpointIdCounter,
- completedCheckpoints,
- savepointStore,
- checkpointStatsTracker);
- }
-
- // TODO: register this class to execution graph as job status change listeners
-
- // TODO: register client as job / execution status change listeners if they are interested
-
- /*
- TODO: decide whether we should take the savepoint before recovery
-
- if (isRecovery) {
- // this is a recovery of a master failure (this master takes over)
- executionGraph.restoreLatestCheckpointedState();
- } else {
- if (snapshotSettings != null) {
- String savepointPath = snapshotSettings.getSavepointPath();
- if (savepointPath != null) {
- // got a savepoint
- log.info("Starting job from savepoint {}.", savepointPath);
-
- // load the savepoint as a checkpoint into the system
- final CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint(
- jobGraph.getJobID(), executionGraph.getAllVertices(), savepointStore, savepointPath);
- executionGraph.getCheckpointCoordinator().getCheckpointStore().addCheckpoint(savepoint);
-
- // Reset the checkpoint ID counter
- long nextCheckpointId = savepoint.getCheckpointID() + 1;
- log.info("Reset the checkpoint ID to " + nextCheckpointId);
- executionGraph.getCheckpointCoordinator().getCheckpointIdCounter().setCount(nextCheckpointId);
-
- executionGraph.restoreLatestCheckpointedState();
- }
+ // register self as job status change listener
+ executionGraph.registerJobStatusListener(new JobStatusListener() {
+ @Override
+ public void jobStatusChanges(
+ final JobID jobId, final JobStatus newJobStatus, final long timestamp, final Throwable error)
+ {
+ // run in rpc thread to avoid concurrency
+ runAsync(new Runnable() {
+ @Override
+ public void run() {
+ jobStatusChanged(newJobStatus, timestamp, error);
+ }
+ });
}
- }
- */
+ });
- // job is good to go, try to locate resource manager's address
+ // job is ready to go, try to establish connection with resource manager
resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener());
} catch (Throwable t) {
+
+ // TODO - this should not result in a job failure, but another leader should take over
+ // TODO - either this master should retry the execution, or it should relinquish leadership / terminate
+
log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t);
executionGraph.fail(t);
- executionGraph = null;
- final Throwable rt;
+ final JobExecutionException rt;
if (t instanceof JobExecutionException) {
rt = (JobExecutionException) t;
- }
- else {
+ } else {
rt = new JobExecutionException(jobGraph.getJobID(),
- "Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
+ "Failed to start job " + jobGraph.getJobID() + " (" + jobGraph.getName() + ")", t);
}
// TODO: notify client about this failure
@@ -488,34 +335,51 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
executionContext.execute(new Runnable() {
@Override
public void run() {
- if (executionGraph != null) {
- try {
- executionGraph.scheduleForExecution(scheduler);
- } catch (Throwable t) {
- executionGraph.fail(t);
- }
+ try {
+ executionGraph.scheduleForExecution(scheduler);
+ } catch (Throwable t) {
+ executionGraph.fail(t);
}
}
});
}
/**
- * Suspending job, all the running tasks will be cancelled, and runtime data will be cleared.
+ * Suspending job, all the running tasks will be cancelled, and communication with other components
+ * will be disposed.
+ *
+ * <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by
+ * calling the {@link #start(UUID)} method once we take the leadership back again.
*
* @param cause The reason of why this job been suspended.
*/
@RpcMethod
- public void suspendJob(final Throwable cause) {
+ public void suspendExecution(final Throwable cause) {
+ if (leaderSessionID == null) {
+ log.debug("Job has already been suspended or shutdown.");
+ return;
+ }
+
+ // receive no more messages until started again, should be called before we clear self leader id
+ ((StartStoppable) getSelf()).stop();
+
leaderSessionID = null;
+ executionGraph.suspend(cause);
- if (executionGraph != null) {
- executionGraph.suspend(cause);
- executionGraph = null;
+ // disconnect from resource manager:
+ try {
+ resourceManagerLeaderRetriever.stop();
+ } catch (Exception e) {
+ log.warn("Failed to stop resource manager leader retriever when suspending.");
}
+ closeResourceManagerConnection();
+
+ // TODO: disconnect from all registered task managers
- disposeCommunicationWithResourceManager();
}
+ //----------------------------------------------------------------------------------------------
+
/**
* Updates the task execution state for a given task.
*
@@ -523,24 +387,38 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
* @return Acknowledge the task execution state update
*/
@RpcMethod
- public Acknowledge updateTaskExecutionState(final TaskExecutionState taskExecutionState) throws ExecutionGraphException {
+ public Acknowledge updateTaskExecutionState(
+ final UUID leaderSessionID,
+ final TaskExecutionState taskExecutionState) throws Exception
+ {
if (taskExecutionState == null) {
throw new NullPointerException("TaskExecutionState must not be null.");
}
+ if (!this.leaderSessionID.equals(leaderSessionID)) {
+ throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+ + ", actual: " + leaderSessionID);
+ }
+
if (executionGraph.updateState(taskExecutionState)) {
return Acknowledge.get();
} else {
throw new ExecutionGraphException("The execution attempt " +
- taskExecutionState.getID() + " was not found.");
+ taskExecutionState.getID() + " was not found.");
}
}
@RpcMethod
public SerializedInputSplit requestNextInputSplit(
- final JobVertexID vertexID,
- final ExecutionAttemptID executionAttempt) throws Exception
+ final UUID leaderSessionID,
+ final JobVertexID vertexID,
+ final ExecutionAttemptID executionAttempt) throws Exception
{
+ if (!this.leaderSessionID.equals(leaderSessionID)) {
+ throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+ + ", actual: " + leaderSessionID);
+ }
+
final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt);
if (execution == null) {
// can happen when JobManager had already unregistered this execution upon on task failure,
@@ -579,7 +457,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
} catch (Exception ex) {
log.error("Could not serialize the next input split of class {}.", nextInputSplit.getClass(), ex);
IOException reason = new IOException("Could not serialize the next input split of class " +
- nextInputSplit.getClass() + ".", ex);
+ nextInputSplit.getClass() + ".", ex);
vertex.fail(reason);
throw reason;
}
@@ -587,17 +465,31 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
@RpcMethod
public PartitionState requestPartitionState(
- final ResultPartitionID partitionId,
- final ExecutionAttemptID taskExecutionId,
- final IntermediateDataSetID taskResultId)
+ final UUID leaderSessionID,
+ final ResultPartitionID partitionId,
+ final ExecutionAttemptID taskExecutionId,
+ final IntermediateDataSetID taskResultId) throws Exception
{
+ if (!this.leaderSessionID.equals(leaderSessionID)) {
+ throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+ + ", actual: " + leaderSessionID);
+ }
+
final Execution execution = executionGraph.getRegisteredExecutions().get(partitionId.getProducerId());
final ExecutionState state = execution != null ? execution.getState() : null;
return new PartitionState(taskResultId, partitionId.getPartitionId(), state);
}
@RpcMethod
- public Acknowledge scheduleOrUpdateConsumers(final ResultPartitionID partitionID) {
+ public Acknowledge scheduleOrUpdateConsumers(
+ final UUID leaderSessionID,
+ final ResultPartitionID partitionID) throws Exception
+ {
+ if (!this.leaderSessionID.equals(leaderSessionID)) {
+ throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+ + ", actual: " + leaderSessionID);
+ }
+
executionGraph.scheduleOrUpdateConsumers(partitionID);
return Acknowledge.get();
}
@@ -609,223 +501,153 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
@RpcMethod
public void acknowledgeCheckpoint(
- JobID jobID,
- ExecutionAttemptID executionAttemptID,
- long checkpointID,
- CheckpointStateHandles checkpointStateHandles,
- long synchronousDurationMillis,
- long asynchronousDurationMillis,
- long bytesBufferedInAlignment,
- long alignmentDurationNanos) {
- throw new UnsupportedOperationException();
- }
-
- @RpcMethod
- public void declineCheckpoint(
- JobID jobID,
- ExecutionAttemptID executionAttemptID,
- long checkpointID,
- long checkpointTimestamp) {
- throw new UnsupportedOperationException();
- }
-
- @RpcMethod
- public void resourceRemoved(final ResourceID resourceId, final String message) {
- // TODO: remove resource from slot pool
- }
-
- @RpcMethod
- public void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge) {
- if (executionGraph != null) {
- final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
- if (checkpointCoordinator != null) {
- getRpcService().execute(new Runnable() {
- @Override
- public void run() {
- try {
- if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
- log.info("Received message for non-existing checkpoint {}.",
+ final JobID jobID,
+ final ExecutionAttemptID executionAttemptID,
+ final long checkpointID,
+ final CheckpointStateHandles checkpointStateHandles,
+ final long synchronousDurationMillis,
+ final long asynchronousDurationMillis,
+ final long bytesBufferedInAlignment,
+ final long alignmentDurationNanos)
+ {
+ final AcknowledgeCheckpoint acknowledge = new AcknowledgeCheckpoint(
+ jobID,
+ executionAttemptID,
+ checkpointID,
+ checkpointStateHandles,
+ synchronousDurationMillis,
+ asynchronousDurationMillis,
+ bytesBufferedInAlignment,
+ alignmentDurationNanos);
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+
+ if (checkpointCoordinator != null) {
+ getRpcService().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
+ log.info("Received message for non-existing checkpoint {}.",
acknowledge.getCheckpointId());
- }
- } catch (Exception e) {
- log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
}
+ } catch (Exception e) {
+ log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
}
- });
- }
- else {
- log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
- jobGraph.getJobID());
- }
+ }
+ });
} else {
- log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+ log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+ jobGraph.getJobID());
}
}
@RpcMethod
- public void declineCheckpoint(final DeclineCheckpoint decline) {
- if (executionGraph != null) {
- final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
- if (checkpointCoordinator != null) {
- getRpcService().execute(new Runnable() {
- @Override
- public void run() {
- try {
- if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
- log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
- }
- } catch (Exception e) {
- log.error("Error in CheckpointCoordinator while processing {}", decline, e);
+ public void declineCheckpoint(
+ final JobID jobID,
+ final ExecutionAttemptID executionAttemptID,
+ final long checkpointID,
+ final long checkpointTimestamp)
+ {
+ final DeclineCheckpoint decline = new DeclineCheckpoint(
+ jobID, executionAttemptID, checkpointID, checkpointTimestamp);
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+
+ if (checkpointCoordinator != null) {
+ getRpcService().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
+ log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
}
+ } catch (Exception e) {
+ log.error("Error in CheckpointCoordinator while processing {}", decline, e);
}
- });
- } else {
- log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
- jobGraph.getJobID());
- }
+ }
+ });
} else {
- log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+ log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
+ jobGraph.getJobID());
}
}
@RpcMethod
public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
- if (executionGraph != null) {
- if (log.isDebugEnabled()) {
- log.debug("Lookup key-value state for job {} with registration " +
+ if (log.isDebugEnabled()) {
+ log.debug("Lookup key-value state for job {} with registration " +
"name {}.", jobGraph.getJobID(), registrationName);
- }
+ }
- final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
- final KvStateLocation location = registry.getKvStateLocation(registrationName);
- if (location != null) {
- return location;
- } else {
- throw new UnknownKvStateLocation(registrationName);
- }
+ final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
+ final KvStateLocation location = registry.getKvStateLocation(registrationName);
+ if (location != null) {
+ return location;
} else {
- throw new IllegalStateException("Received lookup KvState location request for unavailable job " +
- jobGraph.getJobID());
+ throw new UnknownKvStateLocation(registrationName);
}
}
@RpcMethod
public void notifyKvStateRegistered(
- final JobVertexID jobVertexId,
- final KeyGroupRange keyGroupRange,
- final String registrationName,
- final KvStateID kvStateId,
- final KvStateServerAddress kvStateServerAddress)
+ final JobVertexID jobVertexId,
+ final KeyGroupRange keyGroupRange,
+ final String registrationName,
+ final KvStateID kvStateId,
+ final KvStateServerAddress kvStateServerAddress)
{
- if (executionGraph != null) {
- if (log.isDebugEnabled()) {
- log.debug("Key value state registered for job {} under name {}.",
+ if (log.isDebugEnabled()) {
+ log.debug("Key value state registered for job {} under name {}.",
jobGraph.getJobID(), registrationName);
- }
- try {
- executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
- jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress
- );
- } catch (Exception e) {
- log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
- }
- } else {
- log.error("Received notify KvState registered request for unavailable job " + jobGraph.getJobID());
+ }
+
+ try {
+ executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
+ jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress);
+ } catch (Exception e) {
+ log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
}
}
@RpcMethod
public void notifyKvStateUnregistered(
- JobVertexID jobVertexId,
- KeyGroupRange keyGroupRange,
- String registrationName)
+ JobVertexID jobVertexId,
+ KeyGroupRange keyGroupRange,
+ String registrationName)
{
- if (executionGraph != null) {
- if (log.isDebugEnabled()) {
- log.debug("Key value state unregistered for job {} under name {}.",
+ if (log.isDebugEnabled()) {
+ log.debug("Key value state unregistered for job {} under name {}.",
jobGraph.getJobID(), registrationName);
- }
- try {
- executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
- jobVertexId, keyGroupRange, registrationName
- );
- } catch (Exception e) {
- log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
- }
- } else {
- log.error("Received notify KvState unregistered request for unavailable job " + jobGraph.getJobID());
}
- }
- @RpcMethod
- public Future<TriggerSavepointResponse> triggerSavepoint() throws Exception {
- if (executionGraph != null) {
- final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
- if (checkpointCoordinator != null) {
- try {
- Future<String> savepointFuture = new FlinkFuture<>(
- checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
-
- return savepointFuture.handleAsync(new BiFunction<String, Throwable, TriggerSavepointResponse>() {
- @Override
- public TriggerSavepointResponse apply(String savepointPath, Throwable throwable) {
- if (throwable == null) {
- return new TriggerSavepointResponse.Success(jobGraph.getJobID(), savepointPath);
- }
- else {
- return new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
- new Exception("Failed to complete savepoint", throwable));
- }
- }
- }, getMainThreadExecutor());
-
- } catch (Exception e) {
- FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
- future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
- new Exception("Failed to trigger savepoint", e)));
- return future;
- }
- } else {
- FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
- future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
- new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
- "environment of your job.")));
- return future;
- }
- } else {
- FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
- future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
- new IllegalArgumentException("Received trigger savepoint request for unavailable job " +
- jobGraph.getJobID())));
- return future;
+ try {
+ executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+ jobVertexId, keyGroupRange, registrationName);
+ } catch (Exception e) {
+ log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
}
}
@RpcMethod
- public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
- try {
- log.info("Disposing savepoint at {}.", savepointPath);
-
- // check whether the savepoint exists
- savepointStore.loadSavepoint(savepointPath);
+ public Future<String> triggerSavepoint(final UUID leaderSessionID) throws Exception {
+ if (!this.leaderSessionID.equals(leaderSessionID)) {
+ throw new Exception("Leader id not match, expected: " + this.leaderSessionID
+ + ", actual: " + leaderSessionID);
+ }
- savepointStore.disposeSavepoint(savepointPath);
- return new DisposeSavepointResponse.Success();
- } catch (Exception e) {
- log.error("Failed to dispose savepoint at {}.", savepointPath, e);
- return new DisposeSavepointResponse.Failure(e);
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+ if (checkpointCoordinator != null) {
+ return new FlinkFuture<>(checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
+ } else {
+ throw new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
+ "environment of your job.");
}
}
@RpcMethod
public ClassloadingProps requestClassloadingProps() throws Exception {
- if (executionGraph != null) {
- return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
+ return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
executionGraph.getRequiredJarFiles(),
executionGraph.getRequiredClasspaths());
- } else {
- throw new Exception("Received classloading props request for unavailable job " + jobGraph.getJobID());
- }
}
//----------------------------------------------------------------------------------------------
@@ -838,12 +660,11 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
public void run() {
log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
shutDown();
- jobCompletionActions.onFatalError(cause);
+ errorHandler.onFatalError(cause);
}
});
}
- // TODO - wrap this as StatusListenerMessenger's callback with rpc main thread
private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
final JobID jobID = executionGraph.getJobID();
final String jobName = executionGraph.getJobName();
@@ -871,36 +692,33 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
if (newJobStatus == JobStatus.FINISHED) {
try {
final Map<String, SerializedValue<Object>> accumulatorResults =
- executionGraph.getAccumulatorsSerialized();
+ executionGraph.getAccumulatorsSerialized();
final SerializedJobExecutionResult result = new SerializedJobExecutionResult(
- jobID, 0, accumulatorResults // TODO get correct job duration
+ jobID, 0, accumulatorResults // TODO get correct job duration
);
jobCompletionActions.jobFinished(result.toJobExecutionResult(userCodeLoader));
} catch (Exception e) {
log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e);
final JobExecutionException exception = new JobExecutionException(
- jobID, "Failed to retrieve accumulator results.", e);
+ jobID, "Failed to retrieve accumulator results.", e);
// TODO should we also notify client?
jobCompletionActions.jobFailed(exception);
}
- }
- else if (newJobStatus == JobStatus.CANCELED) {
+ } else if (newJobStatus == JobStatus.CANCELED) {
final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
final JobExecutionException exception = new JobExecutionException(
- jobID, "Job was cancelled.", unpackedError);
+ jobID, "Job was cancelled.", unpackedError);
// TODO should we also notify client?
jobCompletionActions.jobFailed(exception);
- }
- else if (newJobStatus == JobStatus.FAILED) {
+ } else if (newJobStatus == JobStatus.FAILED) {
final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader);
final JobExecutionException exception = new JobExecutionException(
- jobID, "Job execution failed.", unpackedError);
+ jobID, "Job execution failed.", unpackedError);
// TODO should we also notify client?
jobCompletionActions.jobFailed(exception);
- }
- else {
+ } else {
final JobExecutionException exception = new JobExecutionException(
- jobID, newJobStatus + " is not a terminal state.");
+ jobID, newJobStatus + " is not a terminal state.");
// TODO should we also notify client?
jobCompletionActions.jobFailed(exception);
throw new RuntimeException(exception);
@@ -909,7 +727,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
}
private void notifyOfNewResourceManagerLeader(
- final String resourceManagerAddress, final UUID resourceManagerLeaderId)
+ final String resourceManagerAddress, final UUID resourceManagerLeaderId)
{
// IMPORTANT: executed by main thread to avoid concurrence
runAsync(new Runnable() {
@@ -918,17 +736,15 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
if (resourceManagerConnection != null) {
if (resourceManagerAddress != null) {
if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress())
- && resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId()))
- {
+ && resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId())) {
// both address and leader id are not changed, we can keep the old connection
return;
}
log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
- resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
- }
- else {
+ resourceManagerConnection.getTargetAddress(), resourceManagerAddress);
+ } else {
log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
- resourceManagerConnection.getTargetAddress());
+ resourceManagerConnection.getTargetAddress());
}
}
@@ -937,8 +753,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
if (resourceManagerAddress != null) {
log.info("Attempting to register at ResourceManager {}", resourceManagerAddress);
resourceManagerConnection = new ResourceManagerConnection(
- log, jobGraph.getJobID(), leaderSessionID,
- resourceManagerAddress, resourceManagerLeaderId, executionContext);
+ log, jobGraph.getJobID(), leaderSessionID,
+ resourceManagerAddress, resourceManagerLeaderId, executionContext);
resourceManagerConnection.start();
}
}
@@ -952,26 +768,14 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
// TODO - add tests for comment in https://github.com/apache/flink/pull/2565
// verify the response with current connection
if (resourceManagerConnection != null
- && resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
+ && resourceManagerConnection.getTargetLeaderId().equals(success.getResourceManagerLeaderId())) {
log.info("JobManager successfully registered at ResourceManager, leader id: {}.",
- success.getResourceManagerLeaderId());
+ success.getResourceManagerLeaderId());
}
}
});
}
- private void disposeCommunicationWithResourceManager() {
- // 1. stop the leader retriever so we will not receiving updates anymore
- try {
- resourceManagerLeaderRetriever.stop();
- } catch (Exception e) {
- log.warn("Failed to stop resource manager leader retriever.");
- }
-
- // 2. close current connection with ResourceManager if exists
- closeResourceManagerConnection();
- }
-
private void closeResourceManagerConnection() {
if (resourceManagerConnection != null) {
resourceManagerConnection.close();
@@ -980,34 +784,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
}
//----------------------------------------------------------------------------------------------
- // Helper methods
- //----------------------------------------------------------------------------------------------
-
- /**
- * Converts JobVertexIDs to corresponding ExecutionJobVertexes
- *
- * @param executionGraph The execution graph that holds the relationship
- * @param vertexIDs The vertexIDs need to be converted
- * @return The corresponding ExecutionJobVertexes
- * @throws JobExecutionException
- */
- private static List<ExecutionJobVertex> getExecutionJobVertexWithId(
- final ExecutionGraph executionGraph, final List<JobVertexID> vertexIDs)
- throws JobExecutionException
- {
- final List<ExecutionJobVertex> ret = new ArrayList<>(vertexIDs.size());
- for (JobVertexID vertexID : vertexIDs) {
- final ExecutionJobVertex executionJobVertex = executionGraph.getJobVertex(vertexID);
- if (executionJobVertex == null) {
- throw new JobExecutionException(executionGraph.getJobID(),
- "The snapshot checkpointing settings refer to non-existent vertex " + vertexID);
- }
- ret.add(executionJobVertex);
- }
- return ret;
- }
-
- //----------------------------------------------------------------------------------------------
// Utility classes
//----------------------------------------------------------------------------------------------
@@ -1024,19 +800,19 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
}
private class ResourceManagerConnection
- extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
+ extends RegisteredRpcConnection<ResourceManagerGateway, JobMasterRegistrationSuccess>
{
private final JobID jobID;
private final UUID jobManagerLeaderID;
ResourceManagerConnection(
- final Logger log,
- final JobID jobID,
- final UUID jobManagerLeaderID,
- final String resourceManagerAddress,
- final UUID resourceManagerLeaderID,
- final Executor executor)
+ final Logger log,
+ final JobID jobID,
+ final UUID jobManagerLeaderID,
+ final String resourceManagerAddress,
+ final UUID resourceManagerLeaderID,
+ final Executor executor)
{
super(log, resourceManagerAddress, resourceManagerLeaderID, executor);
this.jobID = checkNotNull(jobID);
@@ -1046,12 +822,12 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
@Override
protected RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() {
return new RetryingRegistration<ResourceManagerGateway, JobMasterRegistrationSuccess>(
- log, getRpcService(), "ResourceManager", ResourceManagerGateway.class,
- getTargetAddress(), getTargetLeaderId())
+ log, getRpcService(), "ResourceManager", ResourceManagerGateway.class,
+ getTargetAddress(), getTargetLeaderId())
{
@Override
protected Future<RegistrationResponse> invokeRegistration(ResourceManagerGateway gateway, UUID leaderId,
- long timeoutMillis) throws Exception
+ long timeoutMillis) throws Exception
{
Time timeout = Time.milliseconds(timeoutMillis);
return gateway.registerJobMaster(leaderId, jobManagerLeaderID, getAddress(), jobID, timeout);
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index 4b51258..fbe3f74 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,6 +18,8 @@
package org.apache.flink.runtime.jobmaster;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.concurrent.Future;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -29,15 +31,11 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
-import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
-import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
-import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
-import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
-import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.query.KvStateID;
import org.apache.flink.runtime.query.KvStateLocation;
import org.apache.flink.runtime.query.KvStateServerAddress;
-import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.state.KvState;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
@@ -49,52 +47,56 @@ import java.util.UUID;
*/
public interface JobMasterGateway extends CheckpointCoordinatorGateway {
- /**
- * Starting the job under the given leader session ID.
- */
- void startJob(final UUID leaderSessionID);
+ // ------------------------------------------------------------------------
+ // Job start and stop methods
+ // ------------------------------------------------------------------------
- /**
- * Suspending job, all the running tasks will be cancelled, and runtime status will be cleared.
- * Should re-submit the job before restarting it.
- *
- * @param cause The reason of why this job been suspended.
- */
- void suspendJob(final Throwable cause);
+ void startJobExecution();
+
+ void suspendExecution(Throwable cause);
+
+ // ------------------------------------------------------------------------
/**
* Updates the task execution state for a given task.
*
+ * @param leaderSessionID The leader id of JobManager
* @param taskExecutionState New task execution state for a given task
* @return Future flag of the task execution state update result
*/
- Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+ Future<Acknowledge> updateTaskExecutionState(
+ final UUID leaderSessionID,
+ final TaskExecutionState taskExecutionState);
/**
* Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender
* as a {@link SerializedInputSplit} message.
*
+ * @param leaderSessionID The leader id of JobManager
* @param vertexID The job vertex id
* @param executionAttempt The execution attempt id
* @return The future of the input split. If there is no further input split, will return an empty object.
*/
Future<SerializedInputSplit> requestNextInputSplit(
- final JobVertexID vertexID,
- final ExecutionAttemptID executionAttempt);
+ final UUID leaderSessionID,
+ final JobVertexID vertexID,
+ final ExecutionAttemptID executionAttempt);
/**
* Requests the current state of the partition.
* The state of a partition is currently bound to the state of the producing execution.
*
+ * @param leaderSessionID The leader id of JobManager
* @param partitionId The partition ID of the partition to request the state of.
* @param taskExecutionId The execution attempt ID of the task requesting the partition state.
* @param taskResultId The input gate ID of the task requesting the partition state.
* @return The future of the partition state
*/
Future<PartitionState> requestPartitionState(
- final ResultPartitionID partitionId,
- final ExecutionAttemptID taskExecutionId,
- final IntermediateDataSetID taskResultId);
+ final UUID leaderSessionID,
+ final ResultPartitionID partitionId,
+ final ExecutionAttemptID taskExecutionId,
+ final IntermediateDataSetID taskResultId);
/**
* Notifies the JobManager about available data for a produced partition.
@@ -105,11 +107,15 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* <p>
* The JobManager then can decide when to schedule the partition consumers of the given session.
*
- * @param partitionID The partition which has already produced data
- * @param timeout before the rpc call fails
+ * @param leaderSessionID The leader id of JobManager
+ * @param partitionID The partition which has already produced data
+ * @param timeout before the rpc call fails
* @return Future acknowledge of the schedule or update operation
*/
- Future<Acknowledge> scheduleOrUpdateConsumers(final ResultPartitionID partitionID, @RpcTimeout final Time timeout);
+ Future<Acknowledge> scheduleOrUpdateConsumers(
+ final UUID leaderSessionID,
+ final ResultPartitionID partitionID,
+ @RpcTimeout final Time timeout);
/**
* Disconnects the given {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} from the
@@ -118,31 +124,6 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* @param resourceID identifying the TaskManager to disconnect
*/
void disconnectTaskManager(ResourceID resourceID);
- void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
-
- /**
- * Notifies the JobManager about the removal of a resource.
- *
- * @param resourceId The ID under which the resource is registered.
- * @param message Optional message with details, for logging and debugging.
- */
-
- void resourceRemoved(final ResourceID resourceId, final String message);
-
- /**
- * Notifies the JobManager that the checkpoint of an individual task is completed.
- *
- * @param acknowledge The acknowledge message of the checkpoint
- */
- void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge);
-
- /**
- * Notifies the JobManager that a checkpoint request could not be heeded.
- * This can happen if a Task is already in RUNNING state but is internally not yet ready to perform checkpoints.
- *
- * @param decline The decline message of the checkpoint
- */
- void declineCheckpoint(final DeclineCheckpoint decline);
/**
* Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
@@ -150,7 +131,7 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* @param registrationName Name under which the KvState has been registered.
* @return Future of the requested {@link KvState} location
*/
- Future<KvStateLocation> lookupKvStateLocation(final String registrationName) throws Exception;
+ Future<KvStateLocation> lookupKvStateLocation(final String registrationName);
/**
* @param jobVertexId JobVertexID the KvState instance belongs to.
@@ -160,11 +141,11 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* @param kvStateServerAddress Server address where to find the KvState instance.
*/
void notifyKvStateRegistered(
- final JobVertexID jobVertexId,
- final KeyGroupRange keyGroupRange,
- final String registrationName,
- final KvStateID kvStateId,
- final KvStateServerAddress kvStateServerAddress);
+ final JobVertexID jobVertexId,
+ final KeyGroupRange keyGroupRange,
+ final String registrationName,
+ final KvStateID kvStateId,
+ final KvStateServerAddress kvStateServerAddress);
/**
* @param jobVertexId JobVertexID the KvState instance belongs to.
@@ -172,24 +153,17 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* @param registrationName Name under which the KvState has been registered.
*/
void notifyKvStateUnregistered(
- JobVertexID jobVertexId,
- KeyGroupRange keyGroupRange,
- String registrationName);
+ JobVertexID jobVertexId,
+ KeyGroupRange keyGroupRange,
+ String registrationName);
/**
* Notifies the JobManager to trigger a savepoint for this job.
*
- * @return Future of the savepoint trigger response.
- */
- Future<TriggerSavepointResponse> triggerSavepoint();
-
- /**
- * Notifies the Jobmanager to dispose specified savepoint.
- *
- * @param savepointPath The path of the savepoint.
- * @return The future of the savepoint disponse response.
+ * @param leaderSessionID The leader id of JobManager
+ * @return The savepoint path
*/
- Future<DisposeSavepointResponse> disposeSavepoint(final String savepointPath);
+ Future<String> triggerSavepoint(final UUID leaderSessionID);
/**
* Request the classloading props of this job.
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
index e8fb5bb..019ccfe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniClusterJobDispatcher.java
@@ -25,6 +25,8 @@ import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
import org.apache.flink.runtime.rpc.RpcService;
import org.slf4j.Logger;
@@ -62,6 +64,9 @@ public class MiniClusterJobDispatcher {
/** al the services that the JobManager needs, such as BLOB service, factories, etc */
private final JobManagerServices jobManagerServices;
+ /** Registry for all metrics in the mini cluster */
+ private final MetricRegistry metricRegistry;
+
/** The number of JobManagers to launch (more than one simulates a high-availability setup) */
private final int numJobManagers;
@@ -86,8 +91,9 @@ public class MiniClusterJobDispatcher {
public MiniClusterJobDispatcher(
Configuration config,
RpcService rpcService,
- HighAvailabilityServices haServices) throws Exception {
- this(config, rpcService, haServices, 1);
+ HighAvailabilityServices haServices,
+ MetricRegistry metricRegistry) throws Exception {
+ this(config, rpcService, haServices, metricRegistry, 1);
}
/**
@@ -106,16 +112,18 @@ public class MiniClusterJobDispatcher {
Configuration config,
RpcService rpcService,
HighAvailabilityServices haServices,
+ MetricRegistry metricRegistry,
int numJobManagers) throws Exception {
checkArgument(numJobManagers >= 1);
this.configuration = checkNotNull(config);
this.rpcService = checkNotNull(rpcService);
this.haServices = checkNotNull(haServices);
+ this.metricRegistry = checkNotNull(metricRegistry);
this.numJobManagers = numJobManagers;
LOG.info("Creating JobMaster services");
- this.jobManagerServices = JobManagerServices.fromConfiguration(config);
+ this.jobManagerServices = JobManagerServices.fromConfiguration(config, haServices);
}
// ------------------------------------------------------------------------
@@ -140,9 +148,8 @@ public class MiniClusterJobDispatcher {
if (runners != null) {
this.runners = null;
- Exception shutdownException = new Exception("The MiniCluster is shutting down");
for (JobManagerRunner runner : runners) {
- runner.shutdown(shutdownException);
+ runner.shutdown();
}
}
}
@@ -171,9 +178,9 @@ public class MiniClusterJobDispatcher {
checkState(!shutdown, "mini cluster is shut down");
checkState(runners == null, "mini cluster can only execute one job at a time");
- OnCompletionActions onJobCompletion = new DetachedFinalizer(numJobManagers);
+ DetachedFinalizer finalizer = new DetachedFinalizer(numJobManagers);
- this.runners = startJobRunners(job, onJobCompletion);
+ this.runners = startJobRunners(job, finalizer, finalizer);
}
}
@@ -191,17 +198,17 @@ public class MiniClusterJobDispatcher {
checkNotNull(job);
LOG.info("Received job for blocking execution {} ({})", job.getName(), job.getJobID());
- final BlockingJobSync onJobCompletion = new BlockingJobSync(job.getJobID(), numJobManagers);
+ final BlockingJobSync sync = new BlockingJobSync(job.getJobID(), numJobManagers);
synchronized (lock) {
checkState(!shutdown, "mini cluster is shut down");
checkState(runners == null, "mini cluster can only execute one job at a time");
- this.runners = startJobRunners(job, onJobCompletion);
+ this.runners = startJobRunners(job, sync, sync);
}
try {
- return onJobCompletion.getResult();
+ return sync.getResult();
}
finally {
// always clear the status for the next job
@@ -209,24 +216,26 @@ public class MiniClusterJobDispatcher {
}
}
- private JobManagerRunner[] startJobRunners(JobGraph job, OnCompletionActions onCompletion) throws JobExecutionException {
+ private JobManagerRunner[] startJobRunners(
+ JobGraph job,
+ OnCompletionActions onCompletion,
+ FatalErrorHandler errorHandler) throws JobExecutionException {
LOG.info("Starting {} JobMaster(s) for job {} ({})", numJobManagers, job.getName(), job.getJobID());
JobManagerRunner[] runners = new JobManagerRunner[numJobManagers];
for (int i = 0; i < numJobManagers; i++) {
try {
runners[i] = new JobManagerRunner(job, configuration,
- rpcService, haServices, jobManagerServices, onCompletion);
+ rpcService, haServices, jobManagerServices, metricRegistry,
+ onCompletion, errorHandler);
runners[i].start();
}
catch (Throwable t) {
// shut down all the ones so far
- Exception shutdownCause = new Exception("Failed to properly start all mini cluster JobManagers", t);
-
for (int k = 0; k <= i; k++) {
try {
if (runners[i] != null) {
- runners[i].shutdown(shutdownCause);
+ runners[i].shutdown();
}
} catch (Throwable ignored) {
// silent shutdown
@@ -244,15 +253,15 @@ public class MiniClusterJobDispatcher {
// test methods to simulate job master failures
// ------------------------------------------------------------------------
- public void killJobMaster(int which) {
- checkArgument(which >= 0 && which < numJobManagers, "no such job master");
- checkState(!shutdown, "mini cluster is shut down");
-
- JobManagerRunner[] runners = this.runners;
- checkState(runners != null, "mini cluster it not executing a job right now");
-
- runners[which].shutdown(new Throwable("kill JobManager"));
- }
+// public void killJobMaster(int which) {
+// checkArgument(which >= 0 && which < numJobManagers, "no such job master");
+// checkState(!shutdown, "mini cluster is shut down");
+//
+// JobManagerRunner[] runners = this.runners;
+// checkState(runners != null, "mini cluster it not executing a job right now");
+//
+// runners[which].shutdown(new Throwable("kill JobManager"));
+// }
// ------------------------------------------------------------------------
// utility classes
@@ -263,7 +272,7 @@ public class MiniClusterJobDispatcher {
* In the case of a high-availability test setup, there may be multiple runners.
* After that, it marks the mini cluster as ready to receive new jobs.
*/
- private class DetachedFinalizer implements OnCompletionActions {
+ private class DetachedFinalizer implements OnCompletionActions, FatalErrorHandler {
private final AtomicInteger numJobManagersToWaitFor;
@@ -308,7 +317,7 @@ public class MiniClusterJobDispatcher {
* That way it is guaranteed that after the blocking job submit call returns,
* the dispatcher is immediately free to accept another job.
*/
- private static class BlockingJobSync implements OnCompletionActions {
+ private static class BlockingJobSync implements OnCompletionActions, FatalErrorHandler {
private final JobID jobId;
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
deleted file mode 100644
index 42bfc71..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
+++ /dev/null
@@ -1,49 +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.jobmaster.message;
-
-import java.io.Serializable;
-
-/**
- * The response of the dispose savepoint request to JobManager.
- */
-public abstract class DisposeSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = 6008792963949369567L;
-
- public static class Success extends DisposeSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = 1572462960008711415L;
- }
-
- public static class Failure extends DisposeSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = -7505308325483022458L;
-
- private final Throwable cause;
-
- public Failure(final Throwable cause) {
- this.cause = cause;
- }
-
- public Throwable getCause() {
- return cause;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
deleted file mode 100644
index 0b0edc5..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
+++ /dev/null
@@ -1,74 +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.jobmaster.message;
-
-import org.apache.flink.api.common.JobID;
-
-import java.io.Serializable;
-
-/**
- * The response of the trigger savepoint request to JobManager.
- */
-public abstract class TriggerSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = 3139327824611807707L;
-
- private final JobID jobID;
-
- public JobID getJobID() {
- return jobID;
- }
-
- public TriggerSavepointResponse(final JobID jobID) {
- this.jobID = jobID;
- }
-
- public static class Success extends TriggerSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = -1100637460388881776L;
-
- private final String savepointPath;
-
- public Success(final JobID jobID, final String savepointPath) {
- super(jobID);
- this.savepointPath = savepointPath;
- }
-
- public String getSavepointPath() {
- return savepointPath;
- }
- }
-
- public static class Failure extends TriggerSavepointResponse implements Serializable {
-
- private static final long serialVersionUID = -1668479003490615139L;
-
- private final Throwable cause;
-
- public Failure(final JobID jobID, final Throwable cause) {
- super(jobID);
- this.cause = cause;
- }
-
- public Throwable getCause() {
- return cause;
- }
- }
-}
-
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index 2052f98..4b9100a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -33,8 +33,8 @@ import java.util.concurrent.TimeUnit;
public interface RpcService {
/**
- * Return the address under which the rpc service can be reached. If the rpc service cannot be
- * contacted remotely, then it will return an empty string.
+ * Return the hostname or host address under which the rpc service can be reached.
+ * If the rpc service cannot be contacted remotely, then it will return an empty string.
*
* @return Address of the rpc service or empty string if local rpc service
*/
http://git-wip-us.apache.org/repos/asf/flink/blob/34fef475/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
index ef62ef1..6fcd082 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java
@@ -26,11 +26,16 @@ import org.apache.flink.runtime.taskmanager.CheckpointResponder;
import org.apache.flink.runtime.taskmanager.TaskManagerActions;
import org.apache.flink.util.Preconditions;
+import java.util.UUID;
+
/**
* Container class for JobManager specific communication utils used by the {@link TaskExecutor}.
*/
public class JobManagerConnection {
+ // Job master leader session id
+ private final UUID jobMasterLeaderId;
+
// Gateway to the job master
private final JobMasterGateway jobMasterGateway;
@@ -50,13 +55,15 @@ public class JobManagerConnection {
private final PartitionStateChecker partitionStateChecker;
public JobManagerConnection(
- JobMasterGateway jobMasterGateway,
- TaskManagerActions taskManagerActions,
- CheckpointResponder checkpointResponder,
- LibraryCacheManager libraryCacheManager,
- ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
- PartitionStateChecker partitionStateChecker) {
-
+ UUID jobMasterLeaderId,
+ JobMasterGateway jobMasterGateway,
+ TaskManagerActions taskManagerActions,
+ CheckpointResponder checkpointResponder,
+ LibraryCacheManager libraryCacheManager,
+ ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
+ PartitionStateChecker partitionStateChecker)
+ {
+ this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId);
this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway);
this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions);
this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder);
@@ -65,6 +72,10 @@ public class JobManagerConnection {
this.partitionStateChecker = Preconditions.checkNotNull(partitionStateChecker);
}
+ public UUID getJobMasterLeaderId() {
+ return jobMasterLeaderId;
+ }
+
public JobMasterGateway getJobManagerGateway() {
return jobMasterGateway;
}
[5/8] flink git commit: [FLINK-4735] [cluster management] Implements
some job execution related RPC calls on the JobManager
Posted by se...@apache.org.
[FLINK-4735] [cluster management] Implements some job execution related RPC calls on the JobManager
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/21b9f16b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/21b9f16b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/21b9f16b
Branch: refs/heads/flip-6
Commit: 21b9f16bb09785f72a7592925d3bb50160636797
Parents: 35a44da
Author: Kurt Young <yk...@gmail.com>
Authored: Tue Oct 4 23:00:22 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Thu Oct 13 16:25:49 2016 +0200
----------------------------------------------------------------------
.../flink/runtime/jobmaster/JobMaster.java | 246 +++++++++++++++++--
.../runtime/jobmaster/JobMasterGateway.java | 93 ++++++-
.../jobmaster/message/ClassloadingProps.java | 68 +++++
.../message/DisposeSavepointResponse.java | 49 ++++
.../message/TriggerSavepointResponse.java | 74 ++++++
5 files changed, 507 insertions(+), 23 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
index 8f3a342..3b8fc97 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -28,6 +28,7 @@ import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
+import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
@@ -39,8 +40,11 @@ import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.client.JobSubmissionException;
import org.apache.flink.runtime.client.SerializedJobExecutionResult;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.concurrent.BiFunction;
import org.apache.flink.runtime.concurrent.Future;
+import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
+import org.apache.flink.runtime.concurrent.impl.FlinkFuture;
+import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -61,10 +65,20 @@ import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator;
import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings;
import org.apache.flink.runtime.jobmanager.OnCompletionActions;
import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
+import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
+import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
+import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateLocationRegistry;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.query.UnknownKvStateLocation;
import org.apache.flink.runtime.registration.RegisteredRpcConnection;
import org.apache.flink.runtime.registration.RegistrationResponse;
import org.apache.flink.runtime.registration.RetryingRegistration;
@@ -72,7 +86,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
import org.apache.flink.runtime.rpc.RpcEndpoint;
import org.apache.flink.runtime.rpc.RpcMethod;
import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.state.CheckpointStateHandles;
+import org.apache.flink.runtime.state.KeyGroupRange;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import org.apache.flink.runtime.util.SerializedThrowable;
import org.apache.flink.util.InstantiationUtil;
@@ -520,22 +534,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
throw new ExecutionGraphException("The execution attempt " +
taskExecutionState.getID() + " was not found.");
}
-
- }
-
- //----------------------------------------------------------------------------------------------\u2028
- // Internal methods\u2028
- // ----------------------------------------------------------------------------------------------\u2028\u2028
-
- private void handleFatalError(final Throwable cause) {
- runAsync(new Runnable() {
- @Override
- public void run() {
- log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
- shutDown();
- jobCompletionActions.onFatalError(cause);
- }
- });
}
@RpcMethod
@@ -631,10 +629,220 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
throw new UnsupportedOperationException();
}
+ @RpcMethod
+ public void resourceRemoved(final ResourceID resourceId, final String message) {
+ // TODO: remove resource from slot pool
+ }
+
+ @RpcMethod
+ public void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge) {
+ if (executionGraph != null) {
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+ if (checkpointCoordinator != null) {
+ getRpcService().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ if (!checkpointCoordinator.receiveAcknowledgeMessage(acknowledge)) {
+ log.info("Received message for non-existing checkpoint {}.",
+ acknowledge.getCheckpointId());
+ }
+ } catch (Exception e) {
+ log.error("Error in CheckpointCoordinator while processing {}", acknowledge, e);
+ }
+ }
+ });
+ }
+ else {
+ log.error("Received AcknowledgeCheckpoint message for job {} with no CheckpointCoordinator",
+ jobGraph.getJobID());
+ }
+ } else {
+ log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+ }
+ }
+
+ @RpcMethod
+ public void declineCheckpoint(final DeclineCheckpoint decline) {
+ if (executionGraph != null) {
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+ if (checkpointCoordinator != null) {
+ getRpcService().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ if (!checkpointCoordinator.receiveDeclineMessage(decline)) {
+ log.info("Received message for non-existing checkpoint {}.", decline.getCheckpointId());
+ }
+ } catch (Exception e) {
+ log.error("Error in CheckpointCoordinator while processing {}", decline, e);
+ }
+ }
+ });
+ } else {
+ log.error("Received DeclineCheckpoint message for job {} with no CheckpointCoordinator",
+ jobGraph.getJobID());
+ }
+ } else {
+ log.error("Received AcknowledgeCheckpoint for unavailable job {}", jobGraph.getJobID());
+ }
+ }
+
+ @RpcMethod
+ public KvStateLocation lookupKvStateLocation(final String registrationName) throws Exception {
+ if (executionGraph != null) {
+ if (log.isDebugEnabled()) {
+ log.debug("Lookup key-value state for job {} with registration " +
+ "name {}.", jobGraph.getJobID(), registrationName);
+ }
+
+ final KvStateLocationRegistry registry = executionGraph.getKvStateLocationRegistry();
+ final KvStateLocation location = registry.getKvStateLocation(registrationName);
+ if (location != null) {
+ return location;
+ } else {
+ throw new UnknownKvStateLocation(registrationName);
+ }
+ } else {
+ throw new IllegalStateException("Received lookup KvState location request for unavailable job " +
+ jobGraph.getJobID());
+ }
+ }
+
+ @RpcMethod
+ public void notifyKvStateRegistered(
+ final JobVertexID jobVertexId,
+ final KeyGroupRange keyGroupRange,
+ final String registrationName,
+ final KvStateID kvStateId,
+ final KvStateServerAddress kvStateServerAddress)
+ {
+ if (executionGraph != null) {
+ if (log.isDebugEnabled()) {
+ log.debug("Key value state registered for job {} under name {}.",
+ jobGraph.getJobID(), registrationName);
+ }
+ try {
+ executionGraph.getKvStateLocationRegistry().notifyKvStateRegistered(
+ jobVertexId, keyGroupRange, registrationName, kvStateId, kvStateServerAddress
+ );
+ } catch (Exception e) {
+ log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
+ }
+ } else {
+ log.error("Received notify KvState registered request for unavailable job " + jobGraph.getJobID());
+ }
+ }
+
+ @RpcMethod
+ public void notifyKvStateUnregistered(
+ JobVertexID jobVertexId,
+ KeyGroupRange keyGroupRange,
+ String registrationName)
+ {
+ if (executionGraph != null) {
+ if (log.isDebugEnabled()) {
+ log.debug("Key value state unregistered for job {} under name {}.",
+ jobGraph.getJobID(), registrationName);
+ }
+ try {
+ executionGraph.getKvStateLocationRegistry().notifyKvStateUnregistered(
+ jobVertexId, keyGroupRange, registrationName
+ );
+ } catch (Exception e) {
+ log.error("Failed to notify KvStateRegistry about registration {}.", registrationName);
+ }
+ } else {
+ log.error("Received notify KvState unregistered request for unavailable job " + jobGraph.getJobID());
+ }
+ }
+
+ @RpcMethod
+ public Future<TriggerSavepointResponse> triggerSavepoint() throws Exception {
+ if (executionGraph != null) {
+ final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator();
+ if (checkpointCoordinator != null) {
+ try {
+ Future<String> savepointFuture = new FlinkFuture<>(
+ checkpointCoordinator.triggerSavepoint(System.currentTimeMillis()));
+
+ return savepointFuture.handleAsync(new BiFunction<String, Throwable, TriggerSavepointResponse>() {
+ @Override
+ public TriggerSavepointResponse apply(String savepointPath, Throwable throwable) {
+ if (throwable == null) {
+ return new TriggerSavepointResponse.Success(jobGraph.getJobID(), savepointPath);
+ }
+ else {
+ return new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+ new Exception("Failed to complete savepoint", throwable));
+ }
+ }
+ }, getMainThreadExecutor());
+
+ } catch (Exception e) {
+ FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+ future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+ new Exception("Failed to trigger savepoint", e)));
+ return future;
+ }
+ } else {
+ FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+ future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+ new IllegalStateException("Checkpointing disabled. You can enable it via the execution " +
+ "environment of your job.")));
+ return future;
+ }
+ } else {
+ FlinkCompletableFuture<TriggerSavepointResponse> future = new FlinkCompletableFuture<>();
+ future.complete(new TriggerSavepointResponse.Failure(jobGraph.getJobID(),
+ new IllegalArgumentException("Received trigger savepoint request for unavailable job " +
+ jobGraph.getJobID())));
+ return future;
+ }
+ }
+
+ @RpcMethod
+ public DisposeSavepointResponse disposeSavepoint(final String savepointPath) {
+ try {
+ log.info("Disposing savepoint at {}.", savepointPath);
+
+ // check whether the savepoint exists
+ savepointStore.loadSavepoint(savepointPath);
+
+ savepointStore.disposeSavepoint(savepointPath);
+ return new DisposeSavepointResponse.Success();
+ } catch (Exception e) {
+ log.error("Failed to dispose savepoint at {}.", savepointPath, e);
+ return new DisposeSavepointResponse.Failure(e);
+ }
+ }
+
+ @RpcMethod
+ public ClassloadingProps requestClassloadingProps() throws Exception {
+ if (executionGraph != null) {
+ return new ClassloadingProps(libraryCacheManager.getBlobServerPort(),
+ executionGraph.getRequiredJarFiles(),
+ executionGraph.getRequiredClasspaths());
+ } else {
+ throw new Exception("Received classloading props request for unavailable job " + jobGraph.getJobID());
+ }
+ }
+
//----------------------------------------------------------------------------------------------
// Internal methods
//----------------------------------------------------------------------------------------------
+ private void handleFatalError(final Throwable cause) {
+ runAsync(new Runnable() {
+ @Override
+ public void run() {
+ log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause);
+ shutDown();
+ jobCompletionActions.onFatalError(cause);
+ }
+ });
+ }
+
// TODO - wrap this as StatusListenerMessenger's callback with rpc main thread
private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) {
final JobID jobID = executionGraph.getJobID();
http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
index e3e57d4..4b51258 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -18,8 +18,6 @@
package org.apache.flink.runtime.jobmaster;
-import org.apache.flink.api.common.time.Time;
-import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.concurrent.Future;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
@@ -30,8 +28,18 @@ import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.jobmaster.message.ClassloadingProps;
+import org.apache.flink.runtime.jobmaster.message.DisposeSavepointResponse;
+import org.apache.flink.runtime.jobmaster.message.NextInputSplit;
+import org.apache.flink.runtime.jobmaster.message.TriggerSavepointResponse;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.query.KvStateID;
+import org.apache.flink.runtime.query.KvStateLocation;
+import org.apache.flink.runtime.query.KvStateServerAddress;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.state.KeyGroupRange;
+import org.apache.flink.runtime.state.KvState;
import org.apache.flink.runtime.taskmanager.TaskExecutionState;
import java.util.UUID;
@@ -110,4 +118,81 @@ public interface JobMasterGateway extends CheckpointCoordinatorGateway {
* @param resourceID identifying the TaskManager to disconnect
*/
void disconnectTaskManager(ResourceID resourceID);
+ void scheduleOrUpdateConsumers(final ResultPartitionID partitionID);
+
+ /**
+ * Notifies the JobManager about the removal of a resource.
+ *
+ * @param resourceId The ID under which the resource is registered.
+ * @param message Optional message with details, for logging and debugging.
+ */
+
+ void resourceRemoved(final ResourceID resourceId, final String message);
+
+ /**
+ * Notifies the JobManager that the checkpoint of an individual task is completed.
+ *
+ * @param acknowledge The acknowledge message of the checkpoint
+ */
+ void acknowledgeCheckpoint(final AcknowledgeCheckpoint acknowledge);
+
+ /**
+ * Notifies the JobManager that a checkpoint request could not be heeded.
+ * This can happen if a Task is already in RUNNING state but is internally not yet ready to perform checkpoints.
+ *
+ * @param decline The decline message of the checkpoint
+ */
+ void declineCheckpoint(final DeclineCheckpoint decline);
+
+ /**
+ * Requests a {@link KvStateLocation} for the specified {@link KvState} registration name.
+ *
+ * @param registrationName Name under which the KvState has been registered.
+ * @return Future of the requested {@link KvState} location
+ */
+ Future<KvStateLocation> lookupKvStateLocation(final String registrationName) throws Exception;
+
+ /**
+ * @param jobVertexId JobVertexID the KvState instance belongs to.
+ * @param keyGroupRange Key group range the KvState instance belongs to.
+ * @param registrationName Name under which the KvState has been registered.
+ * @param kvStateId ID of the registered KvState instance.
+ * @param kvStateServerAddress Server address where to find the KvState instance.
+ */
+ void notifyKvStateRegistered(
+ final JobVertexID jobVertexId,
+ final KeyGroupRange keyGroupRange,
+ final String registrationName,
+ final KvStateID kvStateId,
+ final KvStateServerAddress kvStateServerAddress);
+
+ /**
+ * @param jobVertexId JobVertexID the KvState instance belongs to.
+ * @param keyGroupRange Key group index the KvState instance belongs to.
+ * @param registrationName Name under which the KvState has been registered.
+ */
+ void notifyKvStateUnregistered(
+ JobVertexID jobVertexId,
+ KeyGroupRange keyGroupRange,
+ String registrationName);
+
+ /**
+ * Notifies the JobManager to trigger a savepoint for this job.
+ *
+ * @return Future of the savepoint trigger response.
+ */
+ Future<TriggerSavepointResponse> triggerSavepoint();
+
+ /**
+ * Notifies the Jobmanager to dispose specified savepoint.
+ *
+ * @param savepointPath The path of the savepoint.
+ * @return The future of the savepoint disponse response.
+ */
+ Future<DisposeSavepointResponse> disposeSavepoint(final String savepointPath);
+
+ /**
+ * Request the classloading props of this job.
+ */
+ Future<ClassloadingProps> requestClassloadingProps();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
new file mode 100644
index 0000000..2d670b4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/ClassloadingProps.java
@@ -0,0 +1,68 @@
+/*
+ * 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.jobmaster.message;
+
+import org.apache.flink.runtime.blob.BlobKey;
+
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+
+/**
+ * The response of classloading props request to JobManager.
+ */
+public class ClassloadingProps implements Serializable {
+
+ private static final long serialVersionUID = -3282341310808511823L;
+
+ private final int blobManagerPort;
+
+ private final List<BlobKey> requiredJarFiles;
+
+ private final List<URL> requiredClasspaths;
+
+ /**
+ * Constructor of ClassloadingProps.
+ *
+ * @param blobManagerPort The port of the blobManager
+ * @param requiredJarFiles The blob keys of the required jar files
+ * @param requiredClasspaths The urls of the required classpaths
+ */
+ public ClassloadingProps(
+ final int blobManagerPort,
+ final List<BlobKey> requiredJarFiles,
+ final List<URL> requiredClasspaths)
+ {
+ this.blobManagerPort = blobManagerPort;
+ this.requiredJarFiles = requiredJarFiles;
+ this.requiredClasspaths = requiredClasspaths;
+ }
+
+ public int getBlobManagerPort() {
+ return blobManagerPort;
+ }
+
+ public List<BlobKey> getRequiredJarFiles() {
+ return requiredJarFiles;
+ }
+
+ public List<URL> getRequiredClasspaths() {
+ return requiredClasspaths;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
new file mode 100644
index 0000000..42bfc71
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/DisposeSavepointResponse.java
@@ -0,0 +1,49 @@
+/*
+ * 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.jobmaster.message;
+
+import java.io.Serializable;
+
+/**
+ * The response of the dispose savepoint request to JobManager.
+ */
+public abstract class DisposeSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = 6008792963949369567L;
+
+ public static class Success extends DisposeSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = 1572462960008711415L;
+ }
+
+ public static class Failure extends DisposeSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = -7505308325483022458L;
+
+ private final Throwable cause;
+
+ public Failure(final Throwable cause) {
+ this.cause = cause;
+ }
+
+ public Throwable getCause() {
+ return cause;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/21b9f16b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
new file mode 100644
index 0000000..0b0edc5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/message/TriggerSavepointResponse.java
@@ -0,0 +1,74 @@
+/*
+ * 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.jobmaster.message;
+
+import org.apache.flink.api.common.JobID;
+
+import java.io.Serializable;
+
+/**
+ * The response of the trigger savepoint request to JobManager.
+ */
+public abstract class TriggerSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = 3139327824611807707L;
+
+ private final JobID jobID;
+
+ public JobID getJobID() {
+ return jobID;
+ }
+
+ public TriggerSavepointResponse(final JobID jobID) {
+ this.jobID = jobID;
+ }
+
+ public static class Success extends TriggerSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = -1100637460388881776L;
+
+ private final String savepointPath;
+
+ public Success(final JobID jobID, final String savepointPath) {
+ super(jobID);
+ this.savepointPath = savepointPath;
+ }
+
+ public String getSavepointPath() {
+ return savepointPath;
+ }
+ }
+
+ public static class Failure extends TriggerSavepointResponse implements Serializable {
+
+ private static final long serialVersionUID = -1668479003490615139L;
+
+ private final Throwable cause;
+
+ public Failure(final JobID jobID, final Throwable cause) {
+ super(jobID);
+ this.cause = cause;
+ }
+
+ public Throwable getCause() {
+ return cause;
+ }
+ }
+}
+