[1/2] flink git commit: [hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable.
Repository: flink Updated Branches: refs/heads/flip-6 fdeda082f -> 3cda59339 [hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/629078ee Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/629078ee Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/629078ee Branch: refs/heads/flip-6 Commit: 629078ee3f5fcecd5498a81abaf8c99f9e614b02 Parents: fdeda08 Author: Stephan Ewen Authored: Wed Sep 21 13:03:17 2016 +0200 Committer: Stephan Ewen Committed: Fri Sep 23 19:44:13 2016 +0200 -- .../flink/runtime/rpc/AsyncCallsTest.java | 4 +- .../flink/runtime/rpc/RpcCompletenessTest.java | 14 ++-- .../flink/runtime/rpc/RpcConnectionTest.java| 86 3 files changed, 96 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/629078ee/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index e8255d4..7affdb9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -43,9 +43,9 @@ public class AsyncCallsTest extends TestLogger { // shared test members // - private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + private static final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static AkkaRpcService akkaRpcService = + private static final AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, Time.milliseconds(1L)); @AfterClass http://git-wip-us.apache.org/repos/asf/flink/blob/629078ee/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index ee3f784..53355e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -30,6 +30,7 @@ import org.reflections.Reflections; import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,7 +70,8 @@ public class RpcCompletenessTest extends TestLogger { @SuppressWarnings("rawtypes") private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { - Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]); + List rpcMethodsFromGateway = getRpcMethodsFromGateway(rpcGateway); + Method[] gatewayMethods = rpcMethodsFromGateway.toArray(new Method[rpcMethodsFromGateway.size()]); Method[] serverMethods = rpcEndpoint.getMethods(); Map> rpcMethods = new HashMap<>(); @@ -360,13 +362,13 @@ public class RpcCompletenessTest extends TestLogger { } // Get all methods declared in current interface - for(Method method : interfaceClass.getDeclaredMethods()) { - allMethods.add(method); - } + Collections.addAll(allMethods, interfaceClass.getDeclaredMethods()); // Get all method inherited from super interface - for(Class superClass : interfaceClass.getInterfaces()) { - allMethods.addAll(getRpcMethodsFromGateway(superClass)); + for (Class superClass : interfaceClass.getInterfaces()) { + @SuppressWarnings("unchecked") + Class gatewayClass = (Class) superClass; + allMethods.addAll(getRpcMethodsFromGateway(gatewayClass)); } return allMethods; } http://git-wip-us.apache.org/repos/asf/flink/blob/629078ee/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/te
[2/2] flink git commit: [FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph
[FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph This closes #2480 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3cda5933 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3cda5933 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3cda5933 Branch: refs/heads/flip-6 Commit: 3cda59339ed593b04be3a897bf04a01e3673db5b Parents: 629078e Author: Kurt Young Authored: Thu Sep 8 12:00:13 2016 +0800 Committer: Stephan Ewen Committed: Fri Sep 23 20:30:05 2016 +0200 -- .../flink/api/common/JobExecutionResult.java| 2 +- .../flink/api/common/JobSubmissionResult.java | 2 +- .../HighAvailabilityServices.java | 12 + .../runtime/highavailability/NonHaServices.java | 16 +- .../runtime/jobmanager/OnCompletionActions.java | 31 ++ .../runtime/jobmanager/scheduler/Scheduler.java | 9 + .../runtime/jobmaster/JobManagerRunner.java | 288 +++ .../runtime/jobmaster/JobManagerServices.java | 73 +++ .../flink/runtime/jobmaster/JobMaster.java | 485 ++- .../runtime/jobmaster/JobMasterGateway.java | 13 + .../jobmaster/MiniClusterJobDispatcher.java | 385 +++ .../flink/runtime/rpc/FatalErrorHandler.java| 24 + .../runtime/taskexecutor/TaskExecutor.java | 12 + .../TestingHighAvailabilityServices.java| 39 +- .../jobmaster/JobManagerRunnerMockTest.java | 254 ++ .../flink/runtime/rpc/RpcConnectionTest.java| 17 +- 16 files changed, 1533 insertions(+), 129 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/3cda5933/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index cb4ecc5..7286cc5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; @Public public class JobExecutionResult extends JobSubmissionResult { - private long netRuntime; + private final long netRuntime; private final Map accumulatorResults; http://git-wip-us.apache.org/repos/asf/flink/blob/3cda5933/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java index c5dc869..b0e7e24 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java @@ -26,7 +26,7 @@ import org.apache.flink.annotation.Public; @Public public class JobSubmissionResult { - private JobID jobID; + private final JobID jobID; public JobSubmissionResult(JobID jobID) { this.jobID = jobID; http://git-wip-us.apache.org/repos/asf/flink/blob/3cda5933/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 7634176..d67e927 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,6 +19,8 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +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; @@ -61,4 +63,14 @@ public interface HighAvailabilityServices { * @param jobID The identifier of the job running the election. */ LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception; + + /** +* Gets the checkpoint recovery factory for the job manager +*/ + CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception; + + /** +* Gets
flink git commit: [FLINK-4672] [taskmanager] Do not decorate Actor Kill messages
Repository: flink Updated Branches: refs/heads/release-1.1 62c666f57 -> caa0fbb21 [FLINK-4672] [taskmanager] Do not decorate Actor Kill messages Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/caa0fbb2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/caa0fbb2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/caa0fbb2 Branch: refs/heads/release-1.1 Commit: caa0fbb2157de56c9bdc4bbf8aedb73df90edede Parents: 62c666f Author: Stephan Ewen Authored: Fri Sep 23 18:42:47 2016 +0200 Committer: Stephan Ewen Committed: Mon Sep 26 11:25:01 2016 +0200 -- .../flink/runtime/taskmanager/TaskManager.scala | 2 +- .../runtime/taskmanager/TaskManagerTest.java| 33 ++-- 2 files changed, 31 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/caa0fbb2/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index a7dd789..8e787bb 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1333,7 +1333,7 @@ class TaskManager( "\n" + "A fatal error occurred, forcing the TaskManager to shut down: " + message, cause) -self ! decorateMessage(Kill) +self ! Kill } override def notifyLeaderAddress(leaderAddress: String, leaderSessionID: UUID): Unit = { http://git-wip-us.apache.org/repos/asf/flink/blob/caa0fbb2/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index ce88c09..1c50265 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.taskmanager; import akka.actor.ActorRef; import akka.actor.ActorSystem; +import akka.actor.Kill; import akka.actor.Props; import akka.japi.Creator; import akka.testkit.JavaTestKit; @@ -55,6 +56,7 @@ import org.apache.flink.runtime.messages.StackTraceSampleMessages.ResponseStackT import org.apache.flink.runtime.messages.StackTraceSampleMessages.ResponseStackTraceSampleSuccess; import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample; import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError; import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.messages.TaskMessages.CancelTask; import org.apache.flink.runtime.messages.TaskMessages.PartitionState; @@ -1369,6 +1371,28 @@ public class TaskManagerTest extends TestLogger { }}; } + @Test + public void testTerminationOnFatalError() { + new JavaTestKit(system){{ + + final ActorGateway taskManager = TestingUtils.createTaskManager( + system, + system.deadLetters(), // no jobmanager + new Configuration(), + true, + false); + + try { + watch(taskManager.actor()); + taskManager.tell(new FatalError("test fatal error", new Exception("something super bad"))); + expectTerminated(d, taskManager.actor()); + } + finally { + taskManager.tell(Kill.getInstance()); + } + }}; + } + // public static class SimpleJobManager extends FlinkUntypedActor { @@ -1549,11 +1573,14 @@ public class TaskManagerTest extends TestLogger { @Override public void invoke() throws Exception { - Object o = new Object(); + final Object o = new Object(); + //noinspection SynchronizationOnLocalVariableOrMethodParameter synchroni
[2/2] flink git commit: [FLINK-4672] [taskmanager] Do not decorate Actor Kill messages
[FLINK-4672] [taskmanager] Do not decorate Actor Kill messages Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6f237cfe Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6f237cfe Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6f237cfe Branch: refs/heads/master Commit: 6f237cfe6f70b5b72fedd3dea6fbeb6c929631e8 Parents: 28ff5a3 Author: Stephan Ewen Authored: Fri Sep 23 18:42:47 2016 +0200 Committer: Stephan Ewen Committed: Mon Sep 26 14:11:05 2016 +0200 -- .../flink/runtime/taskmanager/TaskManager.scala | 2 +- .../runtime/taskmanager/TaskManagerTest.java| 33 ++-- 2 files changed, 31 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6f237cfe/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 9e2feb5..04f3168 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1379,7 +1379,7 @@ class TaskManager( "\n" + "A fatal error occurred, forcing the TaskManager to shut down: " + message, cause) -self ! decorateMessage(Kill) +self ! Kill } override def notifyLeaderAddress(leaderAddress: String, leaderSessionID: UUID): Unit = { http://git-wip-us.apache.org/repos/asf/flink/blob/6f237cfe/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 0e53673..0774fd5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -20,6 +20,7 @@ package org.apache.flink.runtime.taskmanager; import akka.actor.ActorRef; import akka.actor.ActorSystem; +import akka.actor.Kill; import akka.actor.Props; import akka.japi.Creator; import akka.testkit.JavaTestKit; @@ -55,6 +56,7 @@ import org.apache.flink.runtime.messages.StackTraceSampleMessages.ResponseStackT import org.apache.flink.runtime.messages.StackTraceSampleMessages.ResponseStackTraceSampleSuccess; import org.apache.flink.runtime.messages.StackTraceSampleMessages.TriggerStackTraceSample; import org.apache.flink.runtime.messages.TaskManagerMessages; +import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError; import org.apache.flink.runtime.messages.TaskMessages; import org.apache.flink.runtime.messages.TaskMessages.CancelTask; import org.apache.flink.runtime.messages.TaskMessages.PartitionState; @@ -1367,6 +1369,28 @@ public class TaskManagerTest extends TestLogger { }}; } + @Test + public void testTerminationOnFatalError() { + new JavaTestKit(system){{ + + final ActorGateway taskManager = TestingUtils.createTaskManager( + system, + system.deadLetters(), // no jobmanager + new Configuration(), + true, + false); + + try { + watch(taskManager.actor()); + taskManager.tell(new FatalError("test fatal error", new Exception("something super bad"))); + expectTerminated(d, taskManager.actor()); + } + finally { + taskManager.tell(Kill.getInstance()); + } + }}; + } + // public static class SimpleJobManager extends FlinkUntypedActor { @@ -1547,11 +1571,14 @@ public class TaskManagerTest extends TestLogger { @Override public void invoke() throws Exception { - Object o = new Object(); + final Object o = new Object(); + //noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (o) { - o.wait(); + //noins
[1/2] flink git commit: [FLINK-4218] [checkpoints] Do not rely on FileSystem to determing state sizes
Repository: flink Updated Branches: refs/heads/master 28ff5a3c9 -> 95e9004e3 [FLINK-4218] [checkpoints] Do not rely on FileSystem to determing state sizes This prevents failures on eventually consistent S3, where the operations for keys (=entries in the parent directory/bucket) are not guaranteed to be immediately consistent (visible) after a blob was written. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/95e9004e Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/95e9004e Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/95e9004e Branch: refs/heads/master Commit: 95e9004e36fffae755eab7aa3d5d0d5e8bfb7113 Parents: 6f237cf Author: Stephan Ewen Authored: Fri Sep 23 15:16:27 2016 +0200 Committer: Stephan Ewen Committed: Mon Sep 26 14:11:05 2016 +0200 -- .../runtime/checkpoint/CompletedCheckpoint.java | 2 +- .../flink/runtime/checkpoint/TaskState.java | 2 +- .../savepoint/SavepointV1Serializer.java| 6 ++-- .../flink/runtime/state/ChainedStateHandle.java | 2 +- .../runtime/state/KeyGroupsStateHandle.java | 2 +- .../state/RetrievableStreamStateHandle.java | 9 +++--- .../apache/flink/runtime/state/StateObject.java | 6 ++-- .../state/filesystem/FileStateHandle.java | 32 .../filesystem/FsCheckpointStreamFactory.java | 9 +- .../FileSystemStateStorageHelper.java | 15 +++-- ...ZooKeeperCompletedCheckpointStoreITCase.java | 2 +- .../stats/SimpleCheckpointStatsTrackerTest.java | 18 +++ .../state/AbstractCloseableHandleTest.java | 6 ++-- .../tasks/InterruptSensitiveRestoreTest.java| 3 +- 14 files changed, 58 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/95e9004e/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index e412006..7cb3916 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -108,7 +108,7 @@ public class CompletedCheckpoint implements StateObject { } @Override - public long getStateSize() throws Exception { + public long getStateSize() throws IOException { long result = 0L; for (TaskState taskState : taskStates.values()) { http://git-wip-us.apache.org/repos/asf/flink/blob/95e9004e/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java index 9025090..657dd60 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java @@ -152,7 +152,7 @@ public class TaskState implements StateObject { @Override - public long getStateSize() throws Exception { + public long getStateSize() throws IOException { long result = 0L; for (int i = 0; i < parallelism; i++) { http://git-wip-us.apache.org/repos/asf/flink/blob/95e9004e/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java index 8e05b81..f07f44f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java @@ -197,6 +197,7 @@ class SavepointV1Serializer implements SavepointSerializer { } else if (stateHandle instanceof FileStateHandle) { dos.writeByte(FILE_STREAM_STATE_HANDLE); FileStateHandle fileStateHandle = (FileStateHandle) stateHandle; + dos.writeLong(stateHandle.getStateSize()); dos.writeUTF(fileStateHandle.getFilePath().toString()); } else if (stateHandle instanceof ByteStreamStateHandle) { @@ -218,12 +219,13 @@ class Savepoint
flink git commit: [FLINK-4684] [checkpoints] Remove redundant class loader from CheckpointCoordinator
Repository: flink Updated Branches: refs/heads/master 8fa313c39 -> 70e71c161 [FLINK-4684] [checkpoints] Remove redundant class loader from CheckpointCoordinator Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/70e71c16 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/70e71c16 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/70e71c16 Branch: refs/heads/master Commit: 70e71c16177b40c2418e6a8ca0838bf117f6a926 Parents: 8fa313c Author: Stephan Ewen Authored: Mon Sep 26 12:32:10 2016 +0200 Committer: Stephan Ewen Committed: Mon Sep 26 18:05:01 2016 +0200 -- .../checkpoint/CheckpointCoordinator.java | 5 - .../flink/runtime/checkpoint/TaskState.java | 2 +- .../runtime/executiongraph/ExecutionGraph.java | 1 - .../apache/flink/runtime/state/StateUtil.java | 2 +- .../checkpoint/CheckpointCoordinatorTest.java | 101 +++ .../checkpoint/CheckpointStateRestoreTest.java | 7 +- 6 files changed, 38 insertions(+), 80 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/70e71c16/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 fc40911..6a43ddf 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 @@ -112,9 +112,6 @@ public class CheckpointCoordinator { * need to be ascending across job managers. */ private final CheckpointIDCounter checkpointIdCounter; - /** Class loader used to deserialize the state handles (as they may be user-defined) */ - private final ClassLoader userClassLoader; - /** The base checkpoint interval. Actual trigger time may be affected by the * max concurrent checkpoints and minimum-pause values */ private final long baseInterval; @@ -167,7 +164,6 @@ public class CheckpointCoordinator { ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, - ClassLoader userClassLoader, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, SavepointStore savepointStore, @@ -198,7 +194,6 @@ public class CheckpointCoordinator { this.completedCheckpointStore = checkNotNull(completedCheckpointStore); this.savepointStore = checkNotNull(savepointStore); this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); - this.userClassLoader = checkNotNull(userClassLoader); this.statsTracker = checkNotNull(statsTracker); this.timer = new Timer("Checkpoint Timer", true); http://git-wip-us.apache.org/repos/asf/flink/blob/70e71c16/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java index 657dd60..f5e3618 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java @@ -62,7 +62,7 @@ public class TaskState implements StateObject { "Parallelism " + parallelism + " is not smaller or equal to max parallelism " + maxParallelism + "."); this.jobVertexID = jobVertexID; - //preallocate lists of the required size, so that we can randomly set values to indexes + this.subtaskStates = new HashMap<>(parallelism); this.keyGroupsStateHandles = new HashMap<>(parallelism); http://git-wip-us.apache.org/repos/asf/flink/blob/70e71c16/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 c3cf297..7c3fa0b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/f
flink git commit: [FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable
Repository: flink Updated Branches: refs/heads/flip-6 93775cef6 -> ed5c83dc2 [FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable This closes #2520 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ed5c83dc Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ed5c83dc Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ed5c83dc Branch: refs/heads/flip-6 Commit: ed5c83dc2c2a9d46f293b0de01342829e2e598a5 Parents: 93775ce Author: zhuhaifengleon Authored: Mon Sep 26 17:43:44 2016 +0800 Committer: Stephan Ewen Committed: Tue Sep 27 16:55:22 2016 +0200 -- .../JobMasterToResourceManagerConnection.java | 117 +++ .../registration/RegisteredRpcConnection.java | 192 +++ .../runtime/taskexecutor/TaskExecutor.java | 4 +- ...TaskExecutorToResourceManagerConnection.java | 127 +++- .../RegisteredRpcConnectionTest.java| 183 ++ .../registration/RetryingRegistrationTest.java | 6 +- 6 files changed, 519 insertions(+), 110 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/ed5c83dc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java new file mode 100644 index 000..71fce8c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java @@ -0,0 +1,117 @@ +/* + * 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; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +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.RpcService; +import org.apache.flink.runtime.concurrent.Future; + +import org.slf4j.Logger; + +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The connection between a JobMaster and the ResourceManager. + */ +public class JobMasterToResourceManagerConnection + extends RegisteredRpcConnection { + + /** the JobMaster whose connection to the ResourceManager this represents */ + private final JobMaster jobMaster; + + private final JobID jobID; + + private final UUID jobMasterLeaderId; + + public JobMasterToResourceManagerConnection( + Logger log, + JobID jobID, + JobMaster jobMaster, + UUID jobMasterLeaderId, + String resourceManagerAddress, + UUID resourceManagerLeaderId, + Executor executor) { + + super(log, resourceManagerAddress, resourceManagerLeaderId, executor); + this.jobMaster = checkNotNull(jobMaster); + this.jobID = checkNotNull(jobID); + this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId); + } + + @Override + protected RetryingRegistration generateRegistration() { + return new JobMasterToResourceManagerConnection.ResourceManagerRegistration( + log, jobMaster.getRpcService(), + getTargetAddress(), getTargetLeaderId(), + jobMaster.getAddress(),jobID, jobMasterLeaderId); + } + + @Override + protected void onRegistrationSuccess(JobMasterRegistrationSuccess success) { + } + + @Override +
[1/3] flink git commit: [hotfix] [tests] Speed up streaming state tests by skipping default retry delay.
Repository: flink Updated Branches: refs/heads/master e5d62da2c -> 6e123d287 [hotfix] [tests] Speed up streaming state tests by skipping default retry delay. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6e123d28 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6e123d28 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6e123d28 Branch: refs/heads/master Commit: 6e123d287443430bf1721952c5692069e41d95cc Parents: b1642e3 Author: Stephan Ewen Authored: Tue Sep 27 14:53:57 2016 +0200 Committer: Stephan Ewen Committed: Tue Sep 27 14:58:41 2016 +0200 -- .../flink/test/checkpointing/StreamFaultToleranceTestBase.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6e123d28/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java -- diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java index 5f6cd4a..10f78d4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java @@ -18,13 +18,14 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.TestLogger; + import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -98,6 +99,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger { env.setParallelism(PARALLELISM); env.enableCheckpointing(500); env.getConfig().disableSysoutLogging(); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L)); testProgram(env);
[3/3] flink git commit: [FLINK-4696] [core] Limit number of Akka threads in local minicluster setups
[FLINK-4696] [core] Limit number of Akka threads in local minicluster setups Since Flink uses a rather small number of actors, not too many actor dispatcher threads are needed. To prevent mini cluster setups on multi-core CPUs (32 or 64 cores) to spawn too many threads, this limits the number of dispatcher threads for mini clusters. For proper Flink deployments, the threads are not limited by this change. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6ea9284d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6ea9284d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6ea9284d Branch: refs/heads/master Commit: 6ea9284d29ec79576f073441a5de681019720ab0 Parents: e5d62da Author: Stephan Ewen Authored: Tue Sep 27 14:21:20 2016 +0200 Committer: Stephan Ewen Committed: Tue Sep 27 14:58:41 2016 +0200 -- .../apache/flink/runtime/akka/AkkaUtils.scala| 19 +++ .../runtime/minicluster/FlinkMiniCluster.scala | 10 ++ 2 files changed, 25 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6ea9284d/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 7aa75c0..bd3af33 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 @@ -189,6 +189,25 @@ object AkkaUtils { ConfigFactory.parseString(config) } + def testDispatcherConfig: Config = { +val config = + s""" + |akka { + | actor { + |default-dispatcher { + | fork-join-executor { + |parallelism-factor = 1.0 + |parallelism-min = 1 + |parallelism-max = 4 + | } + |} + | } + |} + """.stripMargin + +ConfigFactory.parseString(config) + } + /** * Creates a Akka config for a remote actor system listening on port on the network interface * identified by hostname. http://git-wip-us.apache.org/repos/asf/flink/blob/6ea9284d/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala -- diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala index 0178bd3..a263f66 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala @@ -40,7 +40,6 @@ import org.slf4j.LoggerFactory import scala.concurrent.duration.{Duration, FiniteDuration} import scala.concurrent._ -import scala.concurrent.forkjoin.ForkJoinPool /** * Abstract base class for Flink's mini cluster. The mini cluster starts a @@ -266,17 +265,20 @@ abstract class FlinkMiniCluster( def startResourceManagerActorSystem(index: Int): ActorSystem = { val config = getResourceManagerAkkaConfig(index) -AkkaUtils.createActorSystem(config) +val testConfig = AkkaUtils.testDispatcherConfig.withFallback(config) +AkkaUtils.createActorSystem(testConfig) } def startJobManagerActorSystem(index: Int): ActorSystem = { val config = getJobManagerAkkaConfig(index) -AkkaUtils.createActorSystem(config) +val testConfig = AkkaUtils.testDispatcherConfig.withFallback(config) +AkkaUtils.createActorSystem(testConfig) } def startTaskManagerActorSystem(index: Int): ActorSystem = { val config = getTaskManagerAkkaConfig(index) -AkkaUtils.createActorSystem(config) +val testConfig = AkkaUtils.testDispatcherConfig.withFallback(config) +AkkaUtils.createActorSystem(testConfig) } def startJobClientActorSystem(jobID: JobID): ActorSystem = {
[2/3] flink git commit: [FLINK-4685] [checkpoints] Gather sync/async duration and alignment information for task checkpoints
[FLINK-4685] [checkpoints] Gather sync/async duration and alignment information for task checkpoints This adds to each 'acknowledge checkpoint' message - number of bytes buffered during alignment - duration of alignment phase - duration of synchronous part of the operator checkpoint - duration of asynchronous part of the operator checkpoint Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b1642e32 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b1642e32 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b1642e32 Branch: refs/heads/master Commit: b1642e32c2f69c60c2b212260c3479feb66a9165 Parents: 6ea9284 Author: Stephan Ewen Authored: Mon Sep 26 14:10:21 2016 +0200 Committer: Stephan Ewen Committed: Tue Sep 27 14:58:41 2016 +0200 -- .../state/RocksDBAsyncSnapshotTest.java | 19 ++-- .../flink/runtime/execution/Environment.java| 43 +++-- .../runtime/jobgraph/tasks/StatefulTask.java| 27 -- .../checkpoint/AcknowledgeCheckpoint.java | 93 +--- .../ActorGatewayCheckpointResponder.java| 23 ++--- .../taskmanager/CheckpointResponder.java| 29 -- .../runtime/taskmanager/RuntimeEnvironment.java | 27 -- .../jobmanager/JobManagerHARecoveryTest.java| 11 ++- .../operators/testutils/DummyEnvironment.java | 14 +-- .../operators/testutils/MockEnvironment.java| 15 ++-- .../runtime/taskmanager/TaskAsyncCallTest.java | 6 ++ .../streaming/runtime/io/BarrierBuffer.java | 83 +++-- .../streaming/runtime/io/BarrierTracker.java| 44 + .../streaming/runtime/io/BufferSpiller.java | 36 +--- .../runtime/io/CheckpointBarrierHandler.java| 30 +-- .../runtime/io/StreamInputProcessor.java| 24 +++-- .../runtime/io/StreamTwoInputProcessor.java | 16 ++-- .../runtime/tasks/OneInputStreamTask.java | 2 +- .../streaming/runtime/tasks/StreamTask.java | 87 -- .../runtime/tasks/TwoInputStreamTask.java | 2 +- .../streaming/runtime/io/BarrierBufferTest.java | 88 ++ .../runtime/io/BarrierTrackerTest.java | 40 +++-- .../runtime/tasks/OneInputStreamTaskTest.java | 21 +++-- .../runtime/tasks/StreamMockEnvironment.java| 14 +-- 24 files changed, 576 insertions(+), 218 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b1642e32/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index d5b9b46..c0c9ca1 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -136,15 +136,16 @@ public class RocksDBAsyncSnapshotTest { testHarness.bufferSize) { @Override - public void acknowledgeCheckpoint(long checkpointId) { - super.acknowledgeCheckpoint(checkpointId); - } - - @Override - public void acknowledgeCheckpoint(long checkpointId, - ChainedStateHandle chainedStateHandle, - List keyGroupStateHandles) { - super.acknowledgeCheckpoint(checkpointId, chainedStateHandle, keyGroupStateHandles); + public void acknowledgeCheckpoint( + long checkpointId, + ChainedStateHandle chainedStateHandle, + List keyGroupStateHandles, + long synchronousDurationMillis, long asynchronousDurationMillis, + long bytesBufferedInAlignment, long alignmentDurationNanos) { + + super.acknowledgeCheckpoint(checkpointId, chainedStateHandle, keyGroupStateHandles, + synchronousDurationMillis, asynchronousDurationMillis, + bytesBufferedInAlignment, alignmentDurationNanos); // block on the latch, to verify that triggerCheckpoint ret
[1/2] flink git commit: [FLINK-4543] [network] Fix potential deadlock in SpilledSubpartitionViewAsyncIO.
Repository: flink Updated Branches: refs/heads/master f1b5b35f5 -> 90902914a [FLINK-4543] [network] Fix potential deadlock in SpilledSubpartitionViewAsyncIO. The deadlock could occur in cases where the SpilledSubpartitionViewAsyncIO would simultaneously try to release a buffer and encounter an error in another thread. The field of congestion was the listener, which is now replaced by an AtomicReference, removing the necessity to lock in the case of reporting the error. This closes #2444 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/90902914 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/90902914 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/90902914 Branch: refs/heads/master Commit: 90902914ac4b11f9554b67ad49e0d697a0d02f93 Parents: b928935 Author: Stephan Ewen Authored: Wed Aug 31 16:22:34 2016 +0200 Committer: Stephan Ewen Committed: Tue Sep 27 19:46:40 2016 +0200 -- .../SpilledSubpartitionViewAsyncIO.java | 26 .../checkpoint/CheckpointIDCounterTest.java | 4 +-- 2 files changed, 11 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/90902914/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java index daccd28..ca25536 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewAsyncIO.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.util.event.NotificationListener; import java.io.IOException; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -70,7 +71,7 @@ class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView { private final ConcurrentLinkedQueue returnedBuffers = new ConcurrentLinkedQueue(); /** A data availability listener. */ - private NotificationListener registeredListener; + private final AtomicReference registeredListener; /** Error, which has occurred in the I/O thread. */ private volatile IOException errorInIOThread; @@ -108,7 +109,8 @@ class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView { this.parent = checkNotNull(parent); this.bufferProvider = checkNotNull(bufferProvider); this.bufferAvailabilityListener = new BufferProviderCallback(this); - + this.registeredListener = new AtomicReference<>(); + this.asyncFileReader = ioManager.createBufferFileReader(channelId, new IOThreadCallback(this)); if (initialSeekPosition > 0) { @@ -154,14 +156,12 @@ class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView { return false; } - if (registeredListener == null) { - registeredListener = listener; - + if (registeredListener.compareAndSet(null, listener)) { return true; + } else { + throw new IllegalStateException("already registered listener"); } } - - throw new IllegalStateException("Already registered listener."); } @Override @@ -279,8 +279,8 @@ class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView { returnedBuffers.add(buffer); - listener = registeredListener; - registeredListener = null; + // after this, the listener should be null + listener = registeredListener.getAndSet(null); // If this was the last buffer before we reached EOF, set the corresponding flag to // ensure that further buffers are correctly recycled and eventually no further reads @@ -303,13 +303,7 @@ class SpilledSubpartitionViewAsyncIO implements ResultSubpartitionView { errorInIOThread = error; } - final NotificationListener lis
[2/2] flink git commit: [FLINK-4560] [build] Enforcer Java version >= 1.7 via Maven enforcer plugin
[FLINK-4560] [build] Enforcer Java version >= 1.7 via Maven enforcer plugin This closes #2458 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b928935b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b928935b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b928935b Branch: refs/heads/master Commit: b928935b8c5be02b23dd2cb87144ae1ea001278c Parents: f1b5b35 Author: shijinkui Authored: Fri Sep 2 10:46:45 2016 +0800 Committer: Stephan Ewen Committed: Tue Sep 27 19:46:40 2016 +0200 -- pom.xml | 10 +++--- 1 file changed, 7 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b928935b/pom.xml -- diff --git a/pom.xml b/pom.xml index 7e517e9..b2229fb 100644 --- a/pom.xml +++ b/pom.xml @@ -96,6 +96,7 @@ under the License. 1.7.7 18.0 2.3.7 + 1.7 2.0.1 2.10.4 @@ -929,8 +930,8 @@ under the License. maven-compiler-plugin 3.1 - 1.7 - 1.7 + ${java.version} + ${java.version} -Xlint:all @@ -999,7 +1000,7 @@ under the License. org.apache.maven.plugins maven-enforcer-plugin - 1.3.1 + 1.4.1 enforce-maven @@ -1012,6 +1013,9 @@ under the License. [3.0.3,) + + ${java.version} +
[1/2] flink git commit: [FLINK-4708] [build] Properly scope Kerberos Test Cluster dependency for tests
Repository: flink Updated Branches: refs/heads/master 62d4af881 -> 8243138c3 [FLINK-4708] [build] Properly scope Kerberos Test Cluster dependency for tests Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8243138c Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8243138c Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8243138c Branch: refs/heads/master Commit: 8243138c3ffb0849a7150d3c2df56eaffccd8d75 Parents: 84af0b9 Author: Stephan Ewen Authored: Thu Sep 29 11:08:07 2016 +0200 Committer: Stephan Ewen Committed: Thu Sep 29 11:11:54 2016 +0200 -- flink-streaming-connectors/flink-connector-filesystem/pom.xml | 7 --- 1 file changed, 4 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8243138c/flink-streaming-connectors/flink-connector-filesystem/pom.xml -- diff --git a/flink-streaming-connectors/flink-connector-filesystem/pom.xml b/flink-streaming-connectors/flink-connector-filesystem/pom.xml index edf299d..ef7e72b 100644 --- a/flink-streaming-connectors/flink-connector-filesystem/pom.xml +++ b/flink-streaming-connectors/flink-connector-filesystem/pom.xml @@ -125,6 +125,7 @@ under the License. org.apache.hadoop hadoop-minikdc ${minikdc.version} + test @@ -133,9 +134,9 @@ under the License. + https://issues.apache.org/jira/browse/DIRSHARED-134 + Required to pull the Mini-KDC transitive dependency + --> org.apache.felix maven-bundle-plugin
[2/2] flink git commit: [FLINK-4710] [build] Remove Guice Dependency from Hadoop2
[FLINK-4710] [build] Remove Guice Dependency from Hadoop2 This dependency is transitively pulled, but not necessary for the parts of the Hadoop libraries used by Flink. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/84af0b90 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/84af0b90 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/84af0b90 Branch: refs/heads/master Commit: 84af0b90ec81bbe2af73a56cc0483166314a492a Parents: 62d4af8 Author: Stephan Ewen Authored: Wed Sep 28 20:33:20 2016 +0200 Committer: Stephan Ewen Committed: Thu Sep 29 11:11:54 2016 +0200 -- flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/84af0b90/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml -- diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml index a1baba4..8a7b101 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml @@ -621,6 +621,10 @@ under the License. com.sun.jersey.contribs jersey-guice + + com.google.inject + guice +
[03/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index a73f3b2..0ca89ef 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -18,29 +18,35 @@ package org.apache.flink.streaming.api.operators; +import org.apache.commons.io.IOUtils; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.core.fs.FSDataInputStream; -import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collection; +import java.util.concurrent.RunnableFuture; + /** * Base class for all stream operators. Operators that contain a user function should extend the class * {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class). @@ -90,7 +96,12 @@ public abstract class AbstractStreamOperator private transient KeySelector stateKeySelector2; /** Backend for keyed state. This might be empty if we're not on a keyed stream. */ - private transient KeyedStateBackend keyedStateBackend; + private transient AbstractKeyedStateBackend keyedStateBackend; + + /** Operator state backend */ + private transient OperatorStateBackend operatorStateBackend; + + private transient Collection lazyRestoreStateHandles; protected transient MetricGroup metrics; @@ -116,9 +127,14 @@ public abstract class AbstractStreamOperator return metrics; } + @Override + public void restoreState(Collection stateHandles) { + this.lazyRestoreStateHandles = stateHandles; + } + /** * This method is called immediately before any elements are processed, it should contain the -* operator's initialization logic. +* operator's initialization logic, e.g. state initialization. * * The default implementation does nothing. * @@ -126,24 +142,39 @@ public abstract class AbstractStreamOperator */ @Override public void open() throws Exception { + initOperatorState(); + initKeyedState(); + } + + private void initKeyedState() { try { TypeSerializer keySerializer = config.getStateKeySerializer(getUserCodeClassloader()); // create a keyed state backend if there is keyed state, as indicated by the presence of a key serializer if (null != keySerializer) { - ExecutionConfig execConf = container.getEnvironment().getExecutionConfig();; KeyGroupRange subTaskKeyGroupRange = KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex( container.getEnvironment().getTaskInfo().getNumberOfKeyGroups(), container.getEnvironment().getTaskInfo().getNumberOfParallelSubtasks(), container.getIndexInSubtaskGroup()); - keyedStateBackend = container.createKeye
[10/10] flink git commit: [FLINK-4379] [checkpoints] Fix minor bug and improve debug logging
[FLINK-4379] [checkpoints] Fix minor bug and improve debug logging Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6f8f5eb3 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6f8f5eb3 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6f8f5eb3 Branch: refs/heads/master Commit: 6f8f5eb3b9ba07cd3bb4d9f7edd43d4b8862acbe Parents: 53ed6ad Author: Stephan Ewen Authored: Thu Sep 29 21:12:38 2016 +0200 Committer: Stephan Ewen Committed: Fri Sep 30 12:38:46 2016 +0200 -- .../streaming/runtime/tasks/StreamTask.java | 19 +-- 1 file changed, 13 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6f8f5eb3/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 1725eca..88c3ba4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -717,6 +717,13 @@ public abstract class StreamTask> cancelables.registerClosable(asyncCheckpointRunnable); asyncOperationsThreadPool.submit(asyncCheckpointRunnable); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} - finished synchronous part of checkpoint {}." + + "Alignment duration: {} ms, snapshot duration {} ms", + getName(), checkpointId, alignmentDurationNanos / 1_000_000, syncDurationMillis); + } + return true; } else { return false; @@ -998,12 +1005,12 @@ public abstract class StreamTask> final long asyncEndNanos = System.nanoTime(); final long asyncDurationMillis = (asyncEndNanos - asyncStartNanos) / 1_000_000; - if (nonPartitionedStateHandles.isEmpty() && keyedStates.isEmpty()) { - owner.getEnvironment().acknowledgeCheckpoint(checkpointId, + if (nonPartitionedStateHandles.isEmpty() && partitioneableStateHandles.isEmpty() && keyedStates.isEmpty()) { + owner.getEnvironment().acknowledgeCheckpoint( + checkpointId, syncDurationMillies, asyncDurationMillis, bytesBufferedInAlignment, alignmentDurationNanos); - } else { - + } else { CheckpointStateHandles allStateHandles = new CheckpointStateHandles( nonPartitionedStateHandles, partitioneableStateHandles, @@ -1016,8 +1023,8 @@ public abstract class StreamTask> } if (LOG.isDebugEnabled()) { - LOG.debug("Finished asynchronous checkpoints for checkpoint {} on task {}. Returning handles on " + - "keyed states {}.", checkpointId, name, keyedStates); + LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", + owner.getName(), checkpointId, asyncDurationMillis); } } catch (Exception e) {
[09/10] flink git commit: [FLINK-4702] [kafka connector] Commit offsets to Kafka asynchronously and don't block on polls
[FLINK-4702] [kafka connector] Commit offsets to Kafka asynchronously and don't block on polls Letting the Kafka commit block on polls means that 'notifyCheckpointComplete()' may take very long. This is mostly relevant for low-throughput Kafka topics. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/92f4539a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/92f4539a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/92f4539a Branch: refs/heads/master Commit: 92f4539afc714f7dbd293c3ad677b3b5807c6911 Parents: 6f8f5eb Author: Stephan Ewen Authored: Thu Sep 29 18:09:51 2016 +0200 Committer: Stephan Ewen Committed: Fri Sep 30 12:38:46 2016 +0200 -- .../kafka/internal/Kafka09Fetcher.java | 73 +++-- .../connectors/kafka/Kafka09FetcherTest.java| 304 +++ 2 files changed, 355 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/92f4539a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 9c861c9..1da2259 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -37,6 +37,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; @@ -50,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; /** * A fetcher that fetches data from Kafka brokers via the Kafka 0.9 consumer API. @@ -74,18 +76,24 @@ public class Kafka09Fetcher extends AbstractFetcher implem /** The maximum number of milliseconds to wait for a fetch batch */ private final long pollTimeout; - /** Mutex to guard against concurrent access to the non-threadsafe Kafka consumer */ - private final Object consumerLock = new Object(); + /** The next offsets that the main thread should commit */ + private final AtomicReference> nextOffsetsToCommit; + + /** The callback invoked by Kafka once an offset commit is complete */ + private final OffsetCommitCallback offsetCommitCallback; /** Reference to the Kafka consumer, once it is created */ private volatile KafkaConsumer consumer; - + /** Reference to the proxy, forwarding exceptions from the fetch thread to the main thread */ private volatile ExceptionProxy errorHandler; /** Flag to mark the main work loop as alive */ private volatile boolean running = true; + /** Flag tracking whether the latest commit request has completed */ + private volatile boolean commitInProgress; + // public Kafka09Fetcher( @@ -105,6 +113,8 @@ public class Kafka09Fetcher extends AbstractFetcher implem this.runtimeContext = runtimeContext; this.kafkaProperties = kafkaProperties; this.pollTimeout = pollTimeout; + this.nextOffsetsToCommit = new AtomicReference<>(); + this.offsetCommitCallback = new CommitCallback(); // if checkpointing is enabled, we are not automatically committing to Kafka. kafkaProperties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, @@ -203,19 +213,23 @@ public class Kafka09Fetcher extends AbstractFetcher implem // main fetch loop while (running) { + + // check if there is something to commit + final Map toCommit = nextOffsetsToCommit.getAndSet(null); + if (toCommit != null && !commitInProgress) { + // reset the work-to-be committed, so we d
[02/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 2036f69..f638ddd 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -50,6 +50,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.CheckpointStateHandles; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; @@ -317,7 +318,7 @@ public class StreamMockEnvironment implements Environment { @Override public void acknowledgeCheckpoint( long checkpointId, - ChainedStateHandle chainedStateHandle, List keyGroupStateHandles, + CheckpointStateHandles checkpointStateHandles, long synchronousDurationMillis, long asynchronousDurationMillis, long bytesBufferedInAlignment, long alignmentDurationNanos) { } http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java index 430c6de..247edd6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java @@ -24,12 +24,14 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateBackend; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; import org.mockito.invocation.InvocationOnMock; @@ -41,11 +43,12 @@ import java.util.Collections; import java.util.concurrent.RunnableFuture; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.anyInt; +import static org.mockito.Mockito.doAnswer; /** * Extension of {@link OneInputStreamOperatorTestHarness} that allows the operator to get - * a {@link KeyedStateBackend}. + * a {@link AbstractKeyedStateBackend}. * */ public class KeyedOneInputStreamOperatorTestHarness @@ -53,7 +56,7 @@ public class KeyedOneInputStreamOperatorTestHarness // in case the operator creates one we store it here so that we // can snapshot its state - private KeyedStateBackend keyedStateBackend = null; + private AbstractKeyedStateBackend keyedStateBackend = null; // when we restore we keep the state here so that we can call restore // when the operator requests the keyed state backend @@ -114,7 +117,7 @@ public class KeyedOneInputStreamOperatorTestHarness final KeyGroupRange keyGroupRange = (KeyGroupRange) invocationOnMock.getArguments()[2]; if(keyedStateBackend != null) { - keyedStateBackend.close(); + keyedStateBackend.dispose(); } if (restoredKeyedState == null) { @@ -148,7 +151,7 @@ public class KeyedOneInputStreamOperatorTe
[04/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 73e2808..2f21574 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -80,11 +80,11 @@ public abstract class StateBackendTestBase { return getStateBackend().createStreamFactory(new JobID(), "test_op"); } - protected KeyedStateBackend createKeyedBackend(TypeSerializer keySerializer) throws Exception { + protected AbstractKeyedStateBackend createKeyedBackend(TypeSerializer keySerializer) throws Exception { return createKeyedBackend(keySerializer, new DummyEnvironment("test", 1, 0)); } - protected KeyedStateBackend createKeyedBackend(TypeSerializer keySerializer, Environment env) throws Exception { + protected AbstractKeyedStateBackend createKeyedBackend(TypeSerializer keySerializer, Environment env) throws Exception { return createKeyedBackend( keySerializer, 10, @@ -92,7 +92,7 @@ public abstract class StateBackendTestBase { env); } - protected KeyedStateBackend createKeyedBackend( + protected AbstractKeyedStateBackend createKeyedBackend( TypeSerializer keySerializer, int numberOfKeyGroups, KeyGroupRange keyGroupRange, @@ -104,14 +104,15 @@ public abstract class StateBackendTestBase { keySerializer, numberOfKeyGroups, keyGroupRange, - env.getTaskKvStateRegistry()); + env.getTaskKvStateRegistry()) +; } - protected KeyedStateBackend restoreKeyedBackend(TypeSerializer keySerializer, KeyGroupsStateHandle state) throws Exception { + protected AbstractKeyedStateBackend restoreKeyedBackend(TypeSerializer keySerializer, KeyGroupsStateHandle state) throws Exception { return restoreKeyedBackend(keySerializer, state, new DummyEnvironment("test", 1, 0)); } - protected KeyedStateBackend restoreKeyedBackend( + protected AbstractKeyedStateBackend restoreKeyedBackend( TypeSerializer keySerializer, KeyGroupsStateHandle state, Environment env) throws Exception { @@ -123,7 +124,7 @@ public abstract class StateBackendTestBase { env); } - protected KeyedStateBackend restoreKeyedBackend( + protected AbstractKeyedStateBackend restoreKeyedBackend( TypeSerializer keySerializer, int numberOfKeyGroups, KeyGroupRange keyGroupRange, @@ -144,7 +145,7 @@ public abstract class StateBackendTestBase { @SuppressWarnings("unchecked") public void testValueState() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - KeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE); + AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE); ValueStateDescriptor kvId = new ValueStateDescriptor<>("id", String.class, null); kvId.initializeSerializerUnlessSet(new ExecutionConfig()); @@ -195,7 +196,7 @@ public abstract class StateBackendTestBase { assertEquals("u3", state.value()); assertEquals("u3", getSerializedValue(kvState, 3, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)); - backend.close(); + backend.dispose(); backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1); snapshot1.discardState(); @@ -211,7 +212,7 @@ public abstract class StateBackendTestBase { assertEquals("2", restored1.value()); assertEquals("2", getSerializedValue(restoredKvState1, 2, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)); - backend.close(); + backend.dispose(); backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2); snapshot2.discardState(); @@ -230,7 +231,7 @@ public abstract class StateBackendTestBase { assertEquals("u3", restored2.value()); assertEquals("u3", getSer
[05/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 9adaa86..c39e436 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -20,7 +20,9 @@ package org.apache.flink.runtime.checkpoint; import com.google.common.collect.Iterables; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.checkpoint.savepoint.HeapSavepointStore; import org.apache.flink.runtime.checkpoint.stats.DisabledCheckpointStatsTracker; import org.apache.flink.runtime.execution.ExecutionState; @@ -34,21 +36,21 @@ import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.CheckpointStateHandles; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; - import org.junit.Assert; import org.junit.Test; - import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; - import scala.concurrent.ExecutionContext; import scala.concurrent.Future; @@ -56,6 +58,8 @@ import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -1459,7 +1463,7 @@ public class CheckpointCoordinatorTest { maxConcurrentAttempts, new ExecutionVertex[] { triggerVertex }, new ExecutionVertex[] { ackVertex }, - new ExecutionVertex[] { commitVertex }, + new ExecutionVertex[] { commitVertex }, new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2, cl), new HeapSavepointStore(), @@ -1531,7 +1535,7 @@ public class CheckpointCoordinatorTest { maxConcurrentAttempts, // max two concurrent checkpoints new ExecutionVertex[] { triggerVertex }, new ExecutionVertex[] { ackVertex }, - new ExecutionVertex[] { commitVertex }, + new ExecutionVertex[] { commitVertex }, new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2, cl), new HeapSavepointStore(), @@ -1791,29 +1795,29 @@ public class CheckpointCoordinatorTest { for (int index = 0; index < jobVertex1.getParallelism(); index++) { ChainedStateHandle nonPartitionedState = generateStateForVertex(jobVertexID1, index); + ChainedStateHandle partitionableState = generateChainedPartitionableStateHandle(jobVertexID1, index, 2, 8); List partitionedKeyGroupState = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index)); + CheckpointStateHandles checkpointStateHandles = new CheckpointStateHandles(nonPartitionedState, partitionableState, partitionedKeyGroupState); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( - jid, - jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), - checkpointId, -
[07/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java index f5e3618..7e4eded 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskState.java @@ -20,7 +20,9 @@ package org.apache.flink.runtime.checkpoint; import com.google.common.collect.Iterables; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.util.Preconditions; @@ -47,27 +49,36 @@ public class TaskState implements StateObject { /** handles to non-partitioned states, subtaskindex -> subtaskstate */ private final Map subtaskStates; - /** handles to partitioned states, subtaskindex -> keyed state */ + /** handles to partitionable states, subtaskindex -> partitionable state */ + private final Map> partitionableStates; + + /** handles to key-partitioned states, subtaskindex -> keyed state */ private final Map keyGroupsStateHandles; + /** parallelism of the operator when it was checkpointed */ private final int parallelism; /** maximum parallelism of the operator when the job was first created */ private final int maxParallelism; - public TaskState(JobVertexID jobVertexID, int parallelism, int maxParallelism) { + private final int chainLength; + + public TaskState(JobVertexID jobVertexID, int parallelism, int maxParallelism, int chainLength) { Preconditions.checkArgument( parallelism <= maxParallelism, "Parallelism " + parallelism + " is not smaller or equal to max parallelism " + maxParallelism + "."); + Preconditions.checkArgument(chainLength > 0, "There has to be at least one operator in the operator chain."); this.jobVertexID = jobVertexID; this.subtaskStates = new HashMap<>(parallelism); + this.partitionableStates = new HashMap<>(parallelism); this.keyGroupsStateHandles = new HashMap<>(parallelism); this.parallelism = parallelism; this.maxParallelism = maxParallelism; + this.chainLength = chainLength; } public JobVertexID getJobVertexID() { @@ -85,6 +96,20 @@ public class TaskState implements StateObject { } } + public void putPartitionableState( + int subtaskIndex, + ChainedStateHandle partitionableState) { + + Preconditions.checkNotNull(partitionableState); + + if (subtaskIndex < 0 || subtaskIndex >= parallelism) { + throw new IndexOutOfBoundsException("The given sub task index " + subtaskIndex + + " exceeds the maximum number of sub tasks " + subtaskStates.size()); + } else { + partitionableStates.put(subtaskIndex, partitionableState); + } + } + public void putKeyedState(int subtaskIndex, KeyGroupsStateHandle keyGroupsStateHandle) { Preconditions.checkNotNull(keyGroupsStateHandle); @@ -106,6 +131,15 @@ public class TaskState implements StateObject { } } + public ChainedStateHandle getPartitionableState(int subtaskIndex) { + if (subtaskIndex < 0 || subtaskIndex >= parallelism) { + throw new IndexOutOfBoundsException("The given sub task index " + subtaskIndex + + " exceeds the maximum number of sub tasks " + subtaskStates.size()); + } else { + return partitionableStates.get(subtaskIndex); + } + } + public KeyGroupsStateHandle getKeyGroupState(int subtaskIndex) { if (subtaskIndex < 0 || subtaskIndex >= parallelism) { throw new IndexOutOfBoundsException("The given sub task index " + subtaskIndex + @@ -131,6 +165,10 @@ public class TaskState implements StateObject { return maxParallelism; } + public int getChainLength() { + return chainLength; + } + public Collection getKeyGroupStates() { return keyGroupsStateHandles.values(); } @@ -147,
[08/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
[FLINK-4379] [checkpoints] Introduce rescalable operator state This introduces the Operator State Backend, which stores state that is not partitioned by a key. It replaces the 'Checkpointed' interface. Additionally, this introduces CheckpointStateHandles as container for all checkpoint related state handles This closes #2512 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/53ed6ada Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/53ed6ada Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/53ed6ada Branch: refs/heads/master Commit: 53ed6adac8cbe6b5dcb692dc9b94970f3ec5887c Parents: 2afc092 Author: Stefan Richter Authored: Wed Aug 31 23:59:27 2016 +0200 Committer: Stephan Ewen Committed: Fri Sep 30 12:38:46 2016 +0200 -- .../streaming/state/AbstractRocksDBState.java | 6 +- .../state/RocksDBKeyedStateBackend.java | 75 ++-- .../streaming/state/RocksDBStateBackend.java| 8 +- .../state/RocksDBAsyncSnapshotTest.java | 12 +- .../state/RocksDBStateBackendConfigTest.java| 48 ++- .../api/common/functions/RuntimeContext.java| 125 +- .../util/AbstractRuntimeUDFContext.java | 28 +- .../flink/api/common/state/OperatorState.java | 70 --- .../flink/api/common/state/ValueState.java | 2 +- .../java/typeutils/runtime/JavaSerializer.java | 116 + .../flink/hdfstests/FileStateBackendTest.java | 26 +- .../AbstractCEPBasePatternOperator.java | 3 +- .../operator/AbstractCEPPatternOperator.java| 2 - .../AbstractKeyedCEPPatternOperator.java| 2 - .../checkpoint/CheckpointCoordinator.java | 127 +- .../runtime/checkpoint/CompletedCheckpoint.java | 5 - .../checkpoint/OperatorStateRepartitioner.java | 42 ++ .../runtime/checkpoint/PendingCheckpoint.java | 95 +++-- .../RoundRobinOperatorStateRepartitioner.java | 190 + .../flink/runtime/checkpoint/SubtaskState.java | 9 - .../flink/runtime/checkpoint/TaskState.java | 79 +++- .../savepoint/SavepointV1Serializer.java| 97 - .../deployment/TaskDeploymentDescriptor.java| 50 ++- .../flink/runtime/execution/Environment.java| 16 +- .../flink/runtime/executiongraph/Execution.java | 25 +- .../runtime/executiongraph/ExecutionGraph.java | 2 - .../runtime/executiongraph/ExecutionVertex.java | 6 +- .../runtime/jobgraph/tasks/StatefulTask.java| 11 +- .../checkpoint/AcknowledgeCheckpoint.java | 67 ++- .../runtime/state/AbstractCloseableHandle.java | 126 -- .../state/AbstractKeyedStateBackend.java| 342 +++ .../runtime/state/AbstractStateBackend.java | 43 +- .../flink/runtime/state/ChainedStateHandle.java | 7 +- .../runtime/state/CheckpointStateHandles.java | 103 + .../flink/runtime/state/ClosableRegistry.java | 84 .../state/DefaultOperatorStateBackend.java | 215 ++ .../runtime/state/KeyGroupRangeOffsets.java | 2 + .../runtime/state/KeyGroupsStateHandle.java | 6 - .../flink/runtime/state/KeyedStateBackend.java | 301 ++--- .../runtime/state/OperatorStateBackend.java | 35 ++ .../runtime/state/OperatorStateHandle.java | 109 + .../flink/runtime/state/OperatorStateStore.java | 47 +++ ...artitionableCheckpointStateOutputStream.java | 96 + .../state/RetrievableStreamStateHandle.java | 2 +- .../flink/runtime/state/SnapshotProvider.java | 45 ++ .../apache/flink/runtime/state/StateObject.java | 6 +- .../apache/flink/runtime/state/StateUtil.java | 37 -- .../state/filesystem/FileStateHandle.java | 8 +- .../state/filesystem/FsStateBackend.java| 6 +- .../state/heap/HeapKeyedStateBackend.java | 210 - .../state/memory/ByteStreamStateHandle.java | 13 +- .../state/memory/MemoryStateBackend.java| 9 +- .../ActorGatewayCheckpointResponder.java| 11 +- .../taskmanager/CheckpointResponder.java| 15 +- .../runtime/taskmanager/RuntimeEnvironment.java | 12 +- .../apache/flink/runtime/taskmanager/Task.java | 11 +- .../checkpoint/CheckpointCoordinatorTest.java | 421 +++ .../checkpoint/CheckpointStateRestoreTest.java | 46 +- .../CompletedCheckpointStoreTest.java | 2 +- .../checkpoint/PendingCheckpointTest.java | 2 +- .../checkpoint/PendingSavepointTest.java| 2 +- ...ZooKeeperCompletedCheckpointStoreITCase.java | 5 - .../checkpoint/savepoint/SavepointV1Test.java | 20 +- .../stats/SimpleCheckpointStatsTrackerTest.java | 2 +- .../jobmanager/JobManagerHARecoveryTest.java| 20 +- .../messages/CheckpointMessagesTest.java| 17 +- .../operators/testutils/DummyEnvironment.java | 3 +- .../operators/testutils/MockEnvironment.java| 3 +- .../runtime/query/QueryableStateClientTest.java | 4 +- .../runtime
[06/10] flink git commit: [FLINK-4379] [checkpoints] Introduce rescalable operator state
http://git-wip-us.apache.org/repos/asf/flink/blob/53ed6ada/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java index 5612f73..7293a84 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -18,178 +18,55 @@ package org.apache.flink.runtime.state; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.state.FoldingState; -import org.apache.flink.api.common.state.FoldingStateDescriptor; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MergingState; -import org.apache.flink.api.common.state.ReducingState; -import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.State; -import org.apache.flink.api.common.state.StateBackend; import org.apache.flink.api.common.state.StateDescriptor; -import org.apache.flink.api.common.state.ValueState; -import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.util.Preconditions; -import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.concurrent.RunnableFuture; /** - * A keyed state backend is responsible for managing keyed state. The state can be checkpointed - * to streams using {@link #snapshot(long, long, CheckpointStreamFactory)}. + * A keyed state backend provides methods for managing keyed state. * * @param The key by which state is keyed. */ -public abstract class KeyedStateBackend { - - /** {@link TypeSerializer} for our key. */ - protected final TypeSerializer keySerializer; - - /** The currently active key. */ - protected K currentKey; - - /** The key group of the currently active key */ - private int currentKeyGroup; - - /** So that we can give out state when the user uses the same key. */ - protected HashMap> keyValueStatesByName; - - /** For caching the last accessed partitioned state */ - private String lastName; - - @SuppressWarnings("rawtypes") - private KvState lastState; - - /** The number of key-groups aka max parallelism */ - protected final int numberOfKeyGroups; - - /** Range of key-groups for which this backend is responsible */ - protected final KeyGroupRange keyGroupRange; - - /** KvStateRegistry helper for this task */ - protected final TaskKvStateRegistry kvStateRegistry; - - protected final ClassLoader userCodeClassLoader; - - public KeyedStateBackend( - TaskKvStateRegistry kvStateRegistry, - TypeSerializer keySerializer, - ClassLoader userCodeClassLoader, - int numberOfKeyGroups, - KeyGroupRange keyGroupRange) { - - this.kvStateRegistry = Preconditions.checkNotNull(kvStateRegistry); - this.keySerializer = Preconditions.checkNotNull(keySerializer); - this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); - this.numberOfKeyGroups = Preconditions.checkNotNull(numberOfKeyGroups); - this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange); - } +public interface KeyedStateBackend { /** -* Closes the state backend, releasing all internal resources, but does not delete any persistent -* checkpoint data. -* -* @throws Exception Exceptions can be forwarded and will be logged by the system +* Sets the current key that is used for partitioned state. +* @param newKey The new current key. */ - public void close() throws Exception { - if (kvStateRegistry != null) { - kvStateRegistry.unregisterAll(); - } - - lastName = null; - lastState = null; - keyValueStatesByName = null; - } +
[01/10] flink git commit: [FLINK-4573] [web dashboard] Fix potential resource leak due to unclosed RandomAccessFile in TaskManagerLogHandler
Repository: flink Updated Branches: refs/heads/master 477d1c5d4 -> 92f4539af [FLINK-4573] [web dashboard] Fix potential resource leak due to unclosed RandomAccessFile in TaskManagerLogHandler This closes #2556 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2afc0924 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2afc0924 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2afc0924 Branch: refs/heads/master Commit: 2afc092461cf68cf0f3c26a3ab4c58a7bd68cf71 Parents: 477d1c5 Author: Liwei Lin Authored: Tue Sep 27 20:49:52 2016 +0800 Committer: Stephan Ewen Committed: Fri Sep 30 11:32:39 2016 +0200 -- .../webmonitor/handlers/TaskManagerLogHandler.java| 10 +- 1 file changed, 9 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/2afc0924/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java -- diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java index 5343049..2f0d438 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java @@ -210,7 +210,15 @@ public class TaskManagerLogHandler extends RuntimeMonitorHandlerBase { LOG.error("Displaying TaskManager log failed.", e); return; } - long fileLength = raf.length(); + long fileLength; + try { + fileLength = raf.length(); + } catch (IOException ioe) { + display(ctx, request, "Displaying TaskManager log failed."); + LOG.error("Displaying TaskManager log failed.", ioe); + raf.close(); + throw ioe; + } final FileChannel fc = raf.getChannel(); HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
flink git commit: [FLINK-4702] [kafka connector] Commit offsets to Kafka asynchronously and don't block on polls
Repository: flink Updated Branches: refs/heads/release-1.1 caa0fbb21 -> 90d77594f [FLINK-4702] [kafka connector] Commit offsets to Kafka asynchronously and don't block on polls Letting the Kafka commit block on polls means that 'notifyCheckpointComplete()' may take very long. This is mostly relevant for low-throughput Kafka topics. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/90d77594 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/90d77594 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/90d77594 Branch: refs/heads/release-1.1 Commit: 90d77594fffda1d8d15658d363c478ea6430514e Parents: caa0fbb Author: Stephan Ewen Authored: Thu Sep 29 18:09:51 2016 +0200 Committer: Stephan Ewen Committed: Fri Sep 30 12:39:53 2016 +0200 -- .../kafka/internal/Kafka09Fetcher.java | 73 +++-- .../connectors/kafka/Kafka09FetcherTest.java| 304 +++ 2 files changed, 355 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/90d77594/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 9c861c9..1da2259 100644 --- a/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -37,6 +37,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; @@ -50,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; /** * A fetcher that fetches data from Kafka brokers via the Kafka 0.9 consumer API. @@ -74,18 +76,24 @@ public class Kafka09Fetcher extends AbstractFetcher implem /** The maximum number of milliseconds to wait for a fetch batch */ private final long pollTimeout; - /** Mutex to guard against concurrent access to the non-threadsafe Kafka consumer */ - private final Object consumerLock = new Object(); + /** The next offsets that the main thread should commit */ + private final AtomicReference> nextOffsetsToCommit; + + /** The callback invoked by Kafka once an offset commit is complete */ + private final OffsetCommitCallback offsetCommitCallback; /** Reference to the Kafka consumer, once it is created */ private volatile KafkaConsumer consumer; - + /** Reference to the proxy, forwarding exceptions from the fetch thread to the main thread */ private volatile ExceptionProxy errorHandler; /** Flag to mark the main work loop as alive */ private volatile boolean running = true; + /** Flag tracking whether the latest commit request has completed */ + private volatile boolean commitInProgress; + // public Kafka09Fetcher( @@ -105,6 +113,8 @@ public class Kafka09Fetcher extends AbstractFetcher implem this.runtimeContext = runtimeContext; this.kafkaProperties = kafkaProperties; this.pollTimeout = pollTimeout; + this.nextOffsetsToCommit = new AtomicReference<>(); + this.offsetCommitCallback = new CommitCallback(); // if checkpointing is enabled, we are not automatically committing to Kafka. kafkaProperties.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, @@ -203,19 +213,23 @@ public class Kafka09Fetcher extends AbstractFetcher implem // main fetch loop while (running) { + + // check if there is something to commit + final Map toCommit = nextOffsetsToCommit.getAndSet(null); + if (toCommit != null && !commitInProg
[2/2] flink git commit: [FLINK-4744] [streaming api] Followup: Unify names for operator state access methods and comments.
[FLINK-4744] [streaming api] Followup: Unify names for operator state access methods and comments. Also make JavaSerializer package private, as it is not intended for user as a proper TypeSerializer Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/10a42f95 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/10a42f95 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/10a42f95 Branch: refs/heads/master Commit: 10a42f951c5143537c28a0f9df65627e5c632c4b Parents: 56cba7e Author: Stephan Ewen Authored: Wed Oct 5 15:30:26 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../api/common/state/OperatorStateStore.java| 61 ++ .../java/typeutils/runtime/JavaSerializer.java | 119 -- .../state/DefaultOperatorStateBackend.java | 8 +- .../flink/runtime/state/JavaSerializer.java | 122 +++ .../runtime/state/OperatorStateBackend.java | 2 + .../flink/runtime/state/OperatorStateStore.java | 60 - .../runtime/state/OperatorStateBackendTest.java | 15 ++- .../kafka/FlinkKafkaConsumerBase.java | 6 +- .../kafka/FlinkKafkaProducerBase.java | 2 +- .../kafka/AtLeastOnceProducerTest.java | 1 - .../kafka/FlinkKafkaConsumerBaseTest.java | 14 +-- .../api/checkpoint/CheckpointedFunction.java| 4 +- .../operators/AbstractUdfStreamOperator.java| 6 +- .../operators/StreamCheckpointedOperator.java | 5 +- .../streaming/runtime/tasks/StreamTask.java | 2 +- .../runtime/tasks/OneInputStreamTaskTest.java | 4 +- 16 files changed, 217 insertions(+), 214 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/10a42f95/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java new file mode 100644 index 000..03c11f6 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.state; + +import java.io.Serializable; +import java.util.Set; + +/** + * Interface for a backend that manages operator state. + */ +public interface OperatorStateStore { + + String DEFAULT_OPERATOR_STATE_NAME = "_default_"; + + /** +* Creates a state descriptor of the given name that uses Java serialization to persist the +* state. +* +* This is a simple convenience method. For more flexibility on how state serialization +* should happen, use the {@link #getOperatorState(ListStateDescriptor)} method. +* +* @param stateName The name of state to create +* @return A list state using Java serialization to serialize state objects. +* @throws Exception +*/ + ListState getSerializableListState(String stateName) throws Exception; + + /** +* Creates (or restores) a list state. Each state is registered under a unique name. +* The provided serializer is used to de/serialize the state in case of checkpointing (snapshot/restore). +* +* @param stateDescriptor The descriptor for this state, providing a name and serializer. +* @param The generic type of the state +* +* @return A list for all state partitions. +* @throws Exception +*/ +ListState getOperatorState(ListStateDescriptor stateDescriptor) throws Exception; + + /** +* Returns a set with the names of all currently registered states. +* @return set of names for all registered states. +*/ + Set getRegisteredStateNames(); +} http://git-wip-us.apache.org/repos/asf/flink/blob/10a42f95/flink-core/src/main/java/org/apache/flink/
[1/2] flink git commit: [FLINK-4744] [streaming api] Introduce usercode class loader to deserialize partitionable operator state
Repository: flink Updated Branches: refs/heads/master 7c9bc1e51 -> 10a42f951 [FLINK-4744] [streaming api] Introduce usercode class loader to deserialize partitionable operator state This closes #2598 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/56cba7ee Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/56cba7ee Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/56cba7ee Branch: refs/heads/master Commit: 56cba7ee9e00ef4b2493845558b614816b837d1b Parents: 7c9bc1e Author: Stefan Richter Authored: Wed Oct 5 10:35:13 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:12 2016 +0200 -- .../java/typeutils/runtime/JavaSerializer.java | 27 ++- .../apache/flink/util/InstantiationUtil.java| 12 - .../runtime/state/AbstractStateBackend.java | 4 +- .../state/DefaultOperatorStateBackend.java | 16 ++- .../flink/runtime/state/OperatorStateStore.java | 13 ++ .../state/RetrievableStreamStateHandle.java | 2 +- .../zookeeper/ZooKeeperStateHandleStore.java| 2 +- .../checkpoint/CheckpointCoordinatorTest.java | 19 +--- .../jobmanager/JobManagerHARecoveryTest.java| 7 ++- .../runtime/state/OperatorStateBackendTest.java | 15 +-- .../kafka/FlinkKafkaConsumerBase.java | 15 ++- .../kafka/FlinkKafkaConsumerBaseTest.java | 47 ++-- .../api/checkpoint/ListCheckpointed.java| 5 --- .../operators/AbstractUdfStreamOperator.java| 10 ++--- .../runtime/tasks/OneInputStreamTaskTest.java | 8 ++-- .../EventTimeWindowCheckpointingITCase.java | 7 +-- 16 files changed, 118 insertions(+), 91 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/56cba7ee/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaSerializer.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaSerializer.java index 4ae00d1..3af7653 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/JavaSerializer.java @@ -22,16 +22,25 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; public class JavaSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; + private final ClassLoader userClassLoader; + + public JavaSerializer() { + this(Thread.currentThread().getContextClassLoader()); + } + + public JavaSerializer(ClassLoader userClassLoader) { + this.userClassLoader = Preconditions.checkNotNull(userClassLoader); + } + @Override public boolean isImmutableType() { return false; @@ -69,21 +78,15 @@ public class JavaSerializer extends TypeSerializer { @Override public void serialize(T record, DataOutputView target) throws IOException { - ObjectOutputStream oos = new ObjectOutputStream(new DataOutputViewStream(target)); - oos.writeObject(record); - oos.flush(); + InstantiationUtil.serializeObject(new DataOutputViewStream(target), record); } @Override public T deserialize(DataInputView source) throws IOException { - ObjectInputStream ois = new ObjectInputStream(new DataInputViewStream(source)); - try { - @SuppressWarnings("unchecked") - T nfa = (T) ois.readObject(); - return nfa; + return InstantiationUtil.deserializeObject(new DataInputViewStream(source), userClassLoader); } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not deserialize NFA.", e); + throw new IOException("Could not deserialize object.", e); } } @@ -101,7 +104,7 @@ public class JavaSerializer extends TypeSerializer { @Override public boolean equals(Object obj) { - return obj instanceof JavaSerializer && ((JavaSerializer) obj).canEqual(this); + return obj instanceof JavaSerializer && userClassLoader.equals(((JavaSerializer) obj).userClassLoader); }
[06/17] flink git commit: [FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling
[FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling This closes #2546 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8ff451be Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8ff451be Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8ff451be Branch: refs/heads/master Commit: 8ff451bec58e9f5800eb77c74c1d7457b776cc94 Parents: c62776f Author: kl0u Authored: Thu Aug 25 17:38:49 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../state/RocksDBAsyncSnapshotTest.java | 11 +- .../ContinuousFileMonitoringFunctionITCase.java | 17 +- .../hdfstests/ContinuousFileMonitoringTest.java | 209 -- .../fs/bucketing/BucketingSinkTest.java | 4 +- .../source/ContinuousFileReaderOperator.java| 96 --- .../streaming/api/operators/StreamSource.java | 275 +- .../api/operators/StreamSourceContexts.java | 284 +++ .../runtime/tasks/AsyncExceptionHandler.java| 8 +- .../tasks/DefaultTimeServiceProvider.java | 11 +- .../runtime/tasks/OneInputStreamTask.java | 2 +- .../streaming/runtime/tasks/StreamTask.java | 54 +--- .../runtime/tasks/TwoInputStreamTask.java | 2 +- .../operators/StreamSourceOperatorTest.java | 17 +- .../runtime/operators/TimeProviderTest.java | 79 -- ...AlignedProcessingTimeWindowOperatorTest.java | 34 ++- ...AlignedProcessingTimeWindowOperatorTest.java | 36 ++- .../runtime/tasks/StreamMockEnvironment.java| 8 +- .../KeyedOneInputStreamOperatorTestHarness.java | 4 +- .../util/OneInputStreamOperatorTestHarness.java | 23 +- 19 files changed, 694 insertions(+), 480 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java -- diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index bccbabc..2ebd84a 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -86,7 +86,7 @@ public class RocksDBAsyncSnapshotTest { } /** -* This ensures that asynchronous state handles are actually materialized asynchonously. +* This ensures that asynchronous state handles are actually materialized asynchronously. * * We use latches to block at various stages and see if the code still continues through * the parts that are not asynchronous. If the checkpoint is not done asynchronously the @@ -168,7 +168,6 @@ public class RocksDBAsyncSnapshotTest { while (!field.getBoolean(task)) { Thread.sleep(10); } - } } @@ -189,7 +188,9 @@ public class RocksDBAsyncSnapshotTest { Assert.assertTrue(threadPool.awaitTermination(60_000, TimeUnit.MILLISECONDS)); testHarness.waitForTaskCompletion(); - task.checkTimerException(); + if (mockEnv.wasFailedExternally()) { + Assert.fail("Unexpected exception during execution."); + } } /** @@ -261,8 +262,10 @@ public class RocksDBAsyncSnapshotTest { threadPool.shutdown(); Assert.assertTrue(threadPool.awaitTermination(60_000, TimeUnit.MILLISECONDS)); testHarness.waitForTaskCompletion(); - task.checkTimerException(); + if (mockEnv.wasFailedExternally()) { + throw new AsynchronousException(new InterruptedException("Exception was thrown as expected.")); + } Assert.fail("Operation completed. Cancel failed."); } catch (Exception expected) { AsynchronousException asynchronousException = null; http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileMonitoringFunctionITCase.java -- diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFil
[01/17] flink git commit: [FLINK-4737] [core] Ensure that Flink and its Hadoop dependency pull the same version of 'commons-compress'
Repository: flink Updated Branches: refs/heads/master 10a42f951 -> 417c5a4b4 [FLINK-4737] [core] Ensure that Flink and its Hadoop dependency pull the same version of 'commons-compress' Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7a76328d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7a76328d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7a76328d Branch: refs/heads/master Commit: 7a76328d4ec984aaa93d3e3181873254a6b85eed Parents: 81aec41 Author: Stephan Ewen Authored: Tue Oct 4 18:18:09 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- flink-core/pom.xml | 31 +--- .../flink-shaded-hadoop2/pom.xml| 8 + pom.xml | 6 3 files changed, 28 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/flink-core/pom.xml -- diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 40e5a2e..865a253 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -53,6 +53,20 @@ under the License. + + + commons-collections + commons-collections + + + + + + org.apache.commons + commons-compress + + + org.apache.avro @@ -63,26 +77,9 @@ under the License. org.xerial.snappy snappy-java - - org.apache.commons - commons-compress - - - - commons-collections - commons-collections - - - - - org.apache.commons - commons-compress - 1.4 - - org.apache.flink http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml -- diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml index 8a7b101..7df2578 100644 --- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml +++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml @@ -35,6 +35,14 @@ under the License. jar + + + + org.apache.commons + commons-compress + + + org.apache.hadoop hadoop-common http://git-wip-us.apache.org/repos/asf/flink/blob/7a76328d/pom.xml -- diff --git a/pom.xml b/pom.xml index d127ccb..330debe 100644 --- a/pom.xml +++ b/pom.xml @@ -278,6 +278,12 @@ under the License. 3.5 + + org.apache.commons + commons-compress + 1.4.1 + + org.javassist
[08/17] flink git commit: [FLINK-4728] [core, optimizer] Replace reference equality with object equality
[FLINK-4728] [core,optimizer] Replace reference equality with object equality Some cases of testing Integer equality using == rather than Integer.equals(Integer), and some additional cleanup. This closes #2582 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9206b483 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9206b483 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9206b483 Branch: refs/heads/master Commit: 9206b483b68bb41195bdf2da4f0b9c2de517c031 Parents: 10a42f9 Author: Greg Hogan Authored: Mon Oct 3 13:59:57 2016 -0400 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../flink/api/common/operators/Ordering.java| 6 ++--- .../api/common/operators/util/FieldList.java| 6 ++--- .../api/common/io/DelimitedInputFormatTest.java | 1 - .../api/common/io/EnumerateNestedFilesTest.java | 1 - .../api/common/io/FileInputFormatTest.java | 2 -- .../base/OuterJoinOperatorBaseTest.java | 13 - .../typeutils/base/BigDecComparatorTest.java| 2 +- .../typeutils/base/BigDecSerializerTest.java| 1 - .../typeutils/base/DoubleSerializerTest.java| 9 +++ .../typeutils/base/SqlDateComparatorTest.java | 2 +- .../api/java/typeutils/EitherTypeInfoTest.java | 2 +- .../typeutils/runtime/EitherSerializerTest.java | 2 +- .../runtime/TupleComparatorTTT1Test.java| 5 ++-- .../apache/flink/types/CopyableValueTest.java | 6 ++--- .../java/org/apache/flink/types/RecordTest.java | 2 +- .../api/java/functions/FunctionAnnotation.java | 12 - .../api/java/operators/PartitionOperator.java | 2 +- .../apache/flink/api/java/sca/TaggedValue.java | 4 +-- .../java/functions/SemanticPropUtilTest.java| 7 +++-- .../api/java/operator/CrossOperatorTest.java| 16 +-- .../api/java/operator/DistinctOperatorTest.java | 4 +-- .../flink/api/java/operator/GroupingTest.java | 4 +-- .../api/java/operator/JoinOperatorTest.java | 21 +++ .../api/java/operator/MaxByOperatorTest.java| 2 +- .../api/java/operator/MinByOperatorTest.java| 2 +- .../api/java/operator/ReduceOperatorTest.java | 3 --- .../api/java/operator/SortPartitionTest.java| 4 +-- .../translation/DistinctTranslationTest.java| 2 +- .../api/java/sca/UdfAnalyzerExamplesTest.java | 2 +- .../flink/api/java/sca/UdfAnalyzerTest.java | 2 +- .../flink/optimizer/dag/BulkIterationNode.java | 18 ++--- .../optimizer/dag/WorksetIterationNode.java | 2 +- .../dataproperties/GlobalProperties.java| 8 +++--- .../apache/flink/optimizer/plan/PlanNode.java | 16 +-- .../plandump/PlanJSONDumpGenerator.java | 28 ++-- .../optimizer/AdditionalOperatorsTest.java | 2 +- .../optimizer/BranchingPlansCompilerTest.java | 2 +- .../optimizer/PartitioningReusageTest.java | 2 +- .../SemanticPropertiesAPIToPlanTest.java| 2 +- .../optimizer/dag/GroupCombineNodeTest.java | 1 - .../GlobalPropertiesMatchingTest.java | 14 +- 41 files changed, 116 insertions(+), 126 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java index 7332698..afc659a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Ordering.java @@ -18,12 +18,12 @@ package org.apache.flink.api.common.operators; -import java.util.ArrayList; - import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.operators.util.FieldList; import org.apache.flink.api.common.operators.util.FieldSet; +import java.util.ArrayList; + /** * This class represents an ordering on a set of fields. It specifies the fields and order direction * (ascending, descending). @@ -145,7 +145,7 @@ public class Ordering implements Cloneable { } for (int i = 0; i < this.indexes.size(); i++) { - if (this.indexes.get(i) != otherOrdering.indexes.get(i)) { + if (!this.indexes.get(i).equals(otherOrdering.indexes.get(i))) { return false; } http://git-wip-us.apache.org/repos/asf/flink/blob/9206b483/flink-core/src/main/java/org/apache/flink/api/common/operators/util/FieldList.java -
[05/17] flink git commit: [FLINK-4329] [streaming api] Fix Streaming File Source Timestamps/Watermarks Handling
http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java index 247edd6..5275a39 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/KeyedOneInputStreamOperatorTestHarness.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; +import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -92,7 +92,7 @@ public class KeyedOneInputStreamOperatorTestHarness public KeyedOneInputStreamOperatorTestHarness(OneInputStreamOperator operator, ExecutionConfig executionConfig, - TimeServiceProvider testTimeProvider, + TestTimeServiceProvider testTimeProvider, KeySelector keySelector, TypeInformation keyType) { super(operator, executionConfig, testTimeProvider); http://git-wip-us.apache.org/repos/asf/flink/blob/8ff451be/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index d6f46fd..d8a0ee2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -31,15 +31,16 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.streaming.api.operators.StreamCheckpointedOperator; +import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.AsynchronousException; import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider; import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider; import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -87,6 +88,8 @@ public class OneInputStreamOperatorTestHarness { */ private boolean setupCalled = false; + private volatile boolean wasFailedExternally = false; + public OneInputStreamOperatorTestHarness(OneInputStreamOperator operator) { this(operator, new ExecutionConfig()); } @@ -100,7 +103,7 @@ public class OneInputStreamOperatorTestHarness { public OneInputStreamOperatorTestHarness( OneInputStreamOperator operator, ExecutionConfig executionConfig, - TimeServiceProvider testTimeProvider) { + TestTimeServiceProvider testTimeProvider) { this(operator, executionConfig, new Object(), testTimeProvider); } @@ -132,10 +135,10 @@ public class OneInputStreamOperatorTestHarness { doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocation) throws Throwable { - // do nothing + wasFailedExternally = true; return null; } - }).when(mockTask).registerAsyncException(any(AsynchronousException.class)); + }).wh
[16/17] flink git commit: [FLINK-4700] [tests] Expand and harden TimeServiceProvider test
[FLINK-4700] [tests] Expand and harden TimeServiceProvider test Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4fc54e3e Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4fc54e3e Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4fc54e3e Branch: refs/heads/master Commit: 4fc54e3eb341a049529476ef966380d183d099d4 Parents: 8aea8c8 Author: Stephan Ewen Authored: Wed Oct 5 16:44:56 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 20:31:56 2016 +0200 -- .../AbstractFetcherTimestampsTest.java | 2 +- .../runtime/operators/TestTimeProviderTest.java | 113 .../runtime/operators/TimeProviderTest.java | 269 --- ...AlignedProcessingTimeWindowOperatorTest.java | 2 +- ...AlignedProcessingTimeWindowOperatorTest.java | 2 +- .../tasks/DefaultTimeServiceProviderTest.java | 136 +- 6 files changed, 251 insertions(+), 273 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index c3ba7b7..9b5d2e6 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; -import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler; +import org.apache.flink.streaming.runtime.operators.TestTimeProviderTest.ReferenceSettingExceptionHandler; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider; import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; http://git-wip-us.apache.org/repos/asf/flink/blob/4fc54e3e/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java new file mode 100644 index 000..a8f2dc4 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestTimeProviderTest.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.runtime.operators; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.runtime.tasks.AsyncExceptionHandler; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTaskTestHarness; +import org.apache.flink.streaming.runtime.tasks.TestTimeServiceProvider; + +import org.junit.Test; +import org.junit.runner.RunWith; +
[11/17] flink git commit: [FLINK-4739] [elasticsearch connector] Adding packaging details for the Elasticsearch connector
[FLINK-4739] [elasticsearch connector] Adding packaging details for the Elasticsearch connector When an uber-jar containing an Elasticsearch sink is executed, an IllegalArgumentException may occur, which is caused by conflicting files of Elasticsearch and it's dependencies in META-INF/services. This commit adds further clarification to the documentation on how to build a porper uber-jar that can be properly executed by adapting the pom file. This closes #2591 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/beb31fc1 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/beb31fc1 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/beb31fc1 Branch: refs/heads/master Commit: beb31fc1c5a6f5db060e5a4e1d64eacb70632900 Parents: 8000b64 Author: Steffen Hausmann Authored: Mon Oct 3 22:32:01 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:14 2016 +0200 -- docs/dev/connectors/elasticsearch2.md | 32 ++ 1 file changed, 32 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/beb31fc1/docs/dev/connectors/elasticsearch2.md -- diff --git a/docs/dev/connectors/elasticsearch2.md b/docs/dev/connectors/elasticsearch2.md index 8eed690..5f4267e 100644 --- a/docs/dev/connectors/elasticsearch2.md +++ b/docs/dev/connectors/elasticsearch2.md @@ -139,3 +139,35 @@ This now provides a list of Elasticsearch Nodes to which the sink should connect via a `TransportClient`. More information about Elasticsearch can be found [here](https://elastic.co). + + + Packaging the Elasticsearch Connector into an Uber-jar + +For the execution of your Flink program, +it is recommended to build a so-called uber-jar (executable jar) containing all your dependencies +(see [here]({{site.baseurl}}/dev/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution) for further information). + +However, +when an uber-jar containing an Elasticsearch sink is executed, +an `IllegalArgumentException` may occur, +which is caused by conflicting files of Elasticsearch and it's dependencies +in `META-INF/services`: + +``` +IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist. You need to add the corresponding JAR file supporting this SPI to your classpath. The current classpath supports the following names: [es090, completion090, XBloomFilter]] +``` + +If the uber-jar is build by means of maven, +this issue can be avoided by adding the following bits to the pom file: + +``` + +META-INF/services/org.apache.lucene.codecs.Codec + + + META-INF/services/org.apache.lucene.codecs.DocValuesFormat + + + META-INF/services/org.apache.lucene.codecs.PostingsFormat + +```
[10/17] flink git commit: [hotfix] [streaming api] Cleanup watermark initialization in window operator
[hotfix] [streaming api] Cleanup watermark initialization in window operator Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9d24d51f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9d24d51f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9d24d51f Branch: refs/heads/master Commit: 9d24d51f1d19805e23cab370db0f4bbf1c0038bc Parents: dd3416f Author: Stephan Ewen Authored: Tue Oct 4 23:13:53 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:14 2016 +0200 -- .../runtime/operators/windowing/WindowOperator.java | 10 +- 1 file changed, 1 insertion(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9d24d51f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index ffdf334..f010822 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -63,7 +63,6 @@ import org.apache.flink.streaming.runtime.operators.windowing.functions.Internal import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; -import java.io.ObjectInputStream; import java.io.Serializable; import java.util.Collection; import java.util.HashMap; @@ -158,7 +157,7 @@ public class WindowOperator * To keep track of the current watermark so that we can immediately fire if a trigger * registers an event time callback for a timestamp that lies in the past. */ - protected transient long currentWatermark = Long.MIN_VALUE; + protected long currentWatermark = Long.MIN_VALUE; protected transient Context context = new Context(null, null); @@ -214,11 +213,6 @@ public class WindowOperator setChainingStrategy(ChainingStrategy.ALWAYS); } - private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - currentWatermark = -1; - } - @Override @SuppressWarnings("unchecked") public final void setInputType(TypeInformation type, ExecutionConfig executionConfig) { @@ -262,8 +256,6 @@ public class WindowOperator if (windowAssigner instanceof MergingWindowAssigner) { mergingWindowsByKey = new HashMap<>(); } - - currentWatermark = Long.MIN_VALUE; } @Override
[07/17] flink git commit: [FLINK-4737] [core] Add support for bz2 and xy compression in flink-core.
[FLINK-4737] [core] Add support for bz2 and xy compression in flink-core. Adds a dependency on 'commons-compression'. This closes #2002 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/81aec410 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/81aec410 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/81aec410 Branch: refs/heads/master Commit: 81aec4109adc18f4bd4d1ddb91892faed10f4f14 Parents: 954ef08 Author: Milosz Tanski Authored: Wed May 18 00:04:36 2016 -0400 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- docs/dev/batch/index.md | 10 flink-core/pom.xml | 10 +++- .../flink/api/common/io/FileInputFormat.java| 6 ++- .../io/compression/Bzip2InputStreamFactory.java | 50 .../compression/InflaterInputStreamFactory.java | 3 +- .../io/compression/XZInputStreamFactory.java| 49 +++ 6 files changed, 123 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/docs/dev/batch/index.md -- diff --git a/docs/dev/batch/index.md b/docs/dev/batch/index.md index 5cdc36d..0b1c9f9 100644 --- a/docs/dev/batch/index.md +++ b/docs/dev/batch/index.md @@ -1114,6 +1114,16 @@ The following table lists the currently supported compression methods. .gz, .gzip no + + Bzip2 + .bz2 + no + + + XZ + .xz + no + http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/pom.xml -- diff --git a/flink-core/pom.xml b/flink-core/pom.xml index fe24f0e..40e5a2e 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -75,9 +75,15 @@ under the License. commons-collections commons-collections - - + + + org.apache.commons + commons-compress + 1.4 + + + org.apache.flink flink-test-utils-junit http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java index d0f5166..1d092af 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java @@ -19,9 +19,11 @@ package org.apache.flink.api.common.io; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.io.compression.Bzip2InputStreamFactory; import org.apache.flink.api.common.io.compression.DeflateInflaterInputStreamFactory; import org.apache.flink.api.common.io.compression.GzipInflaterInputStreamFactory; import org.apache.flink.api.common.io.compression.InflaterInputStreamFactory; +import org.apache.flink.api.common.io.compression.XZInputStreamFactory; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -116,7 +118,9 @@ public abstract class FileInputFormat extends RichInputFormat[] defaultFactories = { DeflateInflaterInputStreamFactory.getInstance(), - GzipInflaterInputStreamFactory.getInstance() + GzipInflaterInputStreamFactory.getInstance(), + Bzip2InputStreamFactory.getInstance(), + XZInputStreamFactory.getInstance(), }; for (InflaterInputStreamFactory inputStreamFactory : defaultFactories) { for (String fileExtension : inputStreamFactory.getCommonFileExtensions()) { http://git-wip-us.apache.org/repos/asf/flink/blob/81aec410/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java b/flink-core/src/main/java/org/apache/flink/api/common/io/compression/Bzip2InputStreamFactory.java new file mode 100644 index 000..d204907 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/compre
[03/17] flink git commit: [hotfix] [core] Minor code cleanup and correction of javadocs for filesystem input stream classes.
[hotfix] [core] Minor code cleanup and correction of javadocs for filesystem input stream classes. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c62776f0 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c62776f0 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c62776f0 Branch: refs/heads/master Commit: c62776f0f7ac97f6fd484e2a9e0283074d26a444 Parents: 28b37ef Author: Stephan Ewen Authored: Tue Oct 4 13:29:46 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../apache/flink/core/fs/FSDataInputStream.java | 8 .../core/fs/local/LocalDataInputStream.java | 41 +++- .../runtime/fs/hdfs/HadoopDataInputStream.java | 22 +-- 3 files changed, 24 insertions(+), 47 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c62776f0/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java -- diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java index c94a71d..6ce1235 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataInputStream.java @@ -16,13 +16,6 @@ * limitations under the License. */ - -/** - * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache - * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. - */ - package org.apache.flink.core.fs; import org.apache.flink.annotation.Public; @@ -32,7 +25,6 @@ import java.io.InputStream; /** * Interface for a data input stream to a file on a {@link FileSystem}. - * */ @Public public abstract class FSDataInputStream extends InputStream { http://git-wip-us.apache.org/repos/asf/flink/blob/c62776f0/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java -- diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java index 99ca2c4..e7b2828 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataInputStream.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.core.fs.local; import java.io.File; @@ -26,36 +25,31 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FSDataInputStream; +import javax.annotation.Nonnull; + /** * The LocalDataInputStream class is a wrapper class for a data * input stream to the local file system. - * */ @Internal public class LocalDataInputStream extends FSDataInputStream { - /** -* The file input stream used to read data. -*/ - private FileInputStream fis = null; + /** The file input stream used to read data from.*/ + private final FileInputStream fis; /** * Constructs a new LocalDataInputStream object from a given {@link File} object. * -* @param file -*the {@link File} object the data stream is written to -* @throws IOException -* thrown if the data input stream cannot be created +* @param file The File the data stream is read from +* +* @throws IOException Thrown if the data input stream cannot be created. */ - public LocalDataInputStream(final File file) throws IOException { - + public LocalDataInputStream(File file) throws IOException { this.fis = new FileInputStream(file); } - @Override - public void seek(final long desired) throws IOException { - + public void seek(long desired) throws IOException { this.fis.getChannel().position(desired); } @@ -64,37 +58,28 @@ public class LocalDataInputStream extends FSDataInputStream { return this.fis.getChannel().position(); } - @Override public int read() throws IOException { - return this.fis.read(); } - @Override - public int read(final byte[] buffer, final int offset, final int length) throws IOException { - + public int read(@Nonnull byte[] buffer, int offset, int length) throws IOException { return this.fis.read(buffer, offset, length); } -
[15/17] flink git commit: [FLINK-4749] [streaming api] Remove redundant processing time timer sets from window operator
[FLINK-4749] [streaming api] Remove redundant processing time timer sets from window operator Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/47e49774 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/47e49774 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/47e49774 Branch: refs/heads/master Commit: 47e4977442f7045baf29ce7dc772b7f7aff65343 Parents: 9d24d51 Author: Stephan Ewen Authored: Tue Oct 4 23:48:31 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 20:04:34 2016 +0200 -- .../windowing/EvictingWindowOperator.java | 79 +-- .../operators/windowing/WindowOperator.java | 140 +-- .../runtime/tasks/TestTimeServiceProvider.java | 72 +- .../operators/windowing/WindowOperatorTest.java | 24 ++-- 4 files changed, 175 insertions(+), 140 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/47e49774/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java index a838faa..6609e4d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java @@ -262,60 +262,53 @@ public class EvictingWindowOperator extends Window @Override public void trigger(long time) throws Exception { - boolean fire; + Timer timer; - //Remove information about the triggering task - processingTimeTimerFutures.remove(time); - processingTimeTimerTimestamps.remove(time, processingTimeTimerTimestamps.count(time)); + while ((timer = processingTimeTimersQueue.peek()) != null && timer.timestamp <= time) { - do { - Timer timer = processingTimeTimersQueue.peek(); - if (timer != null && timer.timestamp <= time) { - fire = true; + processingTimeTimers.remove(timer); + processingTimeTimersQueue.remove(); - processingTimeTimers.remove(timer); - processingTimeTimersQueue.remove(); + context.key = timer.key; + context.window = timer.window; + setKeyContext(timer.key); - context.key = timer.key; - context.window = timer.window; - setKeyContext(timer.key); - - ListState> windowState; - MergingWindowSet mergingWindows = null; - - if (windowAssigner instanceof MergingWindowAssigner) { - mergingWindows = getMergingWindowSet(); - W stateWindow = mergingWindows.getStateWindow(context.window); - if (stateWindow == null) { - // then the window is already purged and this is a cleanup - // timer set due to allowed lateness that has nothing to clean, - // so it is safe to just ignore - continue; - } - windowState = getPartitionedState(stateWindow, windowSerializer, windowStateDescriptor); - } else { - windowState = getPartitionedState(context.window, windowSerializer, windowStateDescriptor); - } + ListState> windowState; + MergingWindowSet mergingWindows = null; - Iterable> contents = windowState.get(); - if (contents == null) { - // if we have no state, there is nothing to do + if (windowAssigner instanceof MergingWindowAssigner) { + mergingWindows = getMergingWindowSet(); + W stateWindow = mergingWindows.getStateWindow(context.window); + if (state
[12/17] flink git commit: [FLINK-4718] [docs] Fix figure about parallel watermarks.
[FLINK-4718] [docs] Fix figure about parallel watermarks. This closes #2578 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8000b64a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8000b64a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8000b64a Branch: refs/heads/master Commit: 8000b64ae53e096ef557def1b49fc818b05a6d5f Parents: 7a76328 Author: Neil Derraugh Authored: Fri Sep 30 09:12:34 2016 -0400 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:14 2016 +0200 -- docs/fig/parallel_streams_watermarks.svg | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8000b64a/docs/fig/parallel_streams_watermarks.svg -- diff --git a/docs/fig/parallel_streams_watermarks.svg b/docs/fig/parallel_streams_watermarks.svg index f6a4c4b..d3abde8 100644 --- a/docs/fig/parallel_streams_watermarks.svg +++ b/docs/fig/parallel_streams_watermarks.svg @@ -432,7 +432,7 @@ under the License. y="195.85332" id="text3147" xml:space="preserve" - style="font-size:7.50095272px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#00;font-family:Verdana">B|35 + style="font-size:7.50095272px;font-style:italic;font-weight:normal;text-align:start;text-anchor:start;fill:#00;font-family:Verdana">I|35
[13/17] flink git commit: [FLINK-4750] [runtime] Cleanly await end of all currently executing processing time timers when finite streams finish.
[FLINK-4750] [runtime] Cleanly await end of all currently executing processing time timers when finite streams finish. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8aea8c8f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8aea8c8f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8aea8c8f Branch: refs/heads/master Commit: 8aea8c8f427f5511c6064abbc4b85a3ef106743a Parents: 1cd8d4f Author: Stephan Ewen Authored: Wed Oct 5 14:33:01 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 20:04:34 2016 +0200 -- .../tasks/DefaultTimeServiceProvider.java | 151 +++- .../streaming/runtime/tasks/StreamTask.java | 3 + .../runtime/tasks/TestTimeServiceProvider.java | 55 +++--- .../runtime/tasks/TimeServiceProvider.java | 60 +-- .../operators/windowing/NoOpTimerService.java | 7 +- .../tasks/DefaultTimeServiceProviderTest.java | 179 +++ 6 files changed, 414 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/8aea8c8f/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java index 5664eac..d2c743f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/DefaultTimeServiceProvider.java @@ -17,12 +17,20 @@ package org.apache.flink.streaming.runtime.tasks; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.streaming.runtime.operators.Triggerable; +import javax.annotation.Nonnull; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CancellationException; +import java.util.concurrent.Delayed; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -32,6 +40,12 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class DefaultTimeServiceProvider extends TimeServiceProvider { + private static final int STATUS_ALIVE = 0; + private static final int STATUS_QUIESCED = 1; + private static final int STATUS_SHUTDOWN = 2; + + // + /** The containing task that owns this time service provider. */ private final AsyncExceptionHandler task; @@ -41,6 +55,8 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider { /** The executor service that schedules and calls the triggers of this task*/ private final ScheduledThreadPoolExecutor timerService; + private final AtomicInteger status; + public DefaultTimeServiceProvider(AsyncExceptionHandler failureHandler, Object checkpointLock) { this(failureHandler, checkpointLock, null); @@ -50,19 +66,24 @@ public class DefaultTimeServiceProvider extends TimeServiceProvider { AsyncExceptionHandler task, Object checkpointLock, ThreadFactory threadFactory) { - + this.task = checkNotNull(task); this.checkpointLock = checkNotNull(checkpointLock); + this.status = new AtomicInteger(STATUS_ALIVE); + if (threadFactory == null) { this.timerService = new ScheduledThreadPoolExecutor(1); } else { this.timerService = new ScheduledThreadPoolExecutor(1, threadFactory); } - // allow trigger tasks to be removed if all timers for - // that timestamp are removed by user + // tasks should be removed if the future is canceled this.timerService.setRemoveOnCancelPolicy(true); + + // make sure shutdown removes all pending tasks + this.timerService.setContinueExistingPeriodicTasksAfterShutdownPolicy(false); + this.timerService.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); } @Override @@ -73,17 +94,50 @@ public class DefaultTimeServiceProvid
[02/17] flink git commit: [hotfix] Various code cleanups around time service and asynchronous exceptions
[hotfix] Various code cleanups around time service and asynchronous exceptions - DefaultTimeServiceProvider now owns scheduled executor - Enforce that an asynchronous exception handler is always set Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/954ef08f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/954ef08f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/954ef08f Branch: refs/heads/master Commit: 954ef08f374d7e7c1f2b469201b1ea05c6376b33 Parents: 8ff451b Author: Stephan Ewen Authored: Tue Oct 4 16:15:05 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../AbstractFetcherTimestampsTest.java | 122 +++ .../kafka/testutils/MockRuntimeContext.java | 40 +++--- .../api/operators/StreamSourceContexts.java | 6 +- .../runtime/io/StreamInputProcessor.java| 4 +- .../runtime/tasks/AsyncExceptionHandler.java| 1 + .../runtime/tasks/AsynchronousException.java| 11 +- .../tasks/DefaultTimeServiceProvider.java | 57 + .../runtime/tasks/OneInputStreamTask.java | 2 +- .../streaming/runtime/tasks/StreamTask.java | 18 +-- .../runtime/tasks/TestTimeServiceProvider.java | 2 +- .../runtime/tasks/TwoInputStreamTask.java | 2 +- .../runtime/operators/TimeProviderTest.java | 45 +-- ...AlignedProcessingTimeWindowOperatorTest.java | 84 - ...AlignedProcessingTimeWindowOperatorTest.java | 101 +-- .../operators/windowing/NoOpTimerService.java | 49 .../util/OneInputStreamOperatorTestHarness.java | 6 +- 16 files changed, 347 insertions(+), 203 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/954ef08f/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java -- diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java index 8c68fbe..c3ba7b7 100644 --- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java +++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTimestampsTest.java @@ -25,7 +25,10 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext; +import org.apache.flink.streaming.runtime.operators.TimeProviderTest.ReferenceSettingExceptionHandler; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.DefaultTimeServiceProvider; +import org.apache.flink.streaming.runtime.tasks.TimeServiceProvider; import org.apache.flink.util.SerializedValue; import org.junit.Test; @@ -34,6 +37,7 @@ import javax.annotation.Nullable; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import static org.junit.Assert.*; @@ -110,6 +114,7 @@ public class AbstractFetcherTimestampsTest { @Test public void testPeriodicWatermarks() throws Exception { + ExecutionConfig config = new ExecutionConfig(); config.setAutoWatermarkInterval(10); @@ -120,61 +125,70 @@ public class AbstractFetcherTimestampsTest { TestSourceContext sourceContext = new TestSourceContext<>(); - TestFetcher fetcher = new TestFetcher<>( - sourceContext, originalPartitions, - new SerializedValue>(new PeriodicTestExtractor()), - null, new MockRuntimeContext(17, 3, config, sourceContext.getCheckpointLock())); - - final KafkaTopicPartitionState part1 = fetcher.subscribedPartitions()[0]; - final KafkaTopicPartitionState part2 = fetcher.subscribedPartitions()[1]; - final KafkaTopicPartitionState part3 = fetcher.subscribedPartitions()[2]; - - // elements generate a watermark if the timestamp is a multiple of three - - // elements for partition 1 - fetcher.emitRecord(1L, part
[04/17] flink git commit: [FLINK-4709] [core] Fix resource leak in InputStreamFSInputWrapper
[FLINK-4709] [core] Fix resource leak in InputStreamFSInputWrapper This closes #2581 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/28b37ef9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/28b37ef9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/28b37ef9 Branch: refs/heads/master Commit: 28b37ef97d803855e884900d1a983813ea2ac99d Parents: 9206b48 Author: Holger Frydrych Authored: Mon Oct 3 14:34:19 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:13 2016 +0200 -- .../common/io/InputStreamFSInputWrapper.java| 5 +++ .../io/InputStreamFSInputWrapperTest.java | 38 2 files changed, 43 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/28b37ef9/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java index cfd94bc..f7db680 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InputStreamFSInputWrapper.java @@ -42,6 +42,11 @@ public class InputStreamFSInputWrapper extends FSDataInputStream { } @Override + public void close() throws IOException { + this.inStream.close(); + } + + @Override public void seek(long desired) throws IOException { if (desired < this.pos) { throw new IllegalArgumentException("Wrapped InputStream: cannot search backwards."); http://git-wip-us.apache.org/repos/asf/flink/blob/28b37ef9/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java -- diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java new file mode 100644 index 000..8fcd231 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/io/InputStreamFSInputWrapperTest.java @@ -0,0 +1,38 @@ +/* + * 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.api.common.io; + +import java.io.InputStream; + +import org.junit.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class InputStreamFSInputWrapperTest { + + @Test + public void testClose() throws Exception { + InputStream mockedInputStream = mock(InputStream.class); + InputStreamFSInputWrapper wrapper = new InputStreamFSInputWrapper(mockedInputStream); + wrapper.close(); + verify(mockedInputStream).close(); + } + +}
[14/17] flink git commit: [hotfix] [streaming api] Remove obsolete and unused InputTypeSerializer from WindowOperator
[hotfix] [streaming api] Remove obsolete and unused InputTypeSerializer from WindowOperator Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1cd8d4f4 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1cd8d4f4 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1cd8d4f4 Branch: refs/heads/master Commit: 1cd8d4f418a707790c091fed2428627eae9da423 Parents: 47e4977 Author: Stephan Ewen Authored: Tue Oct 4 23:49:54 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 20:04:34 2016 +0200 -- .../operators/windowing/WindowOperator.java | 20 +-- .../windowing/EvictingWindowOperatorTest.java | 6 --- .../operators/windowing/WindowOperatorTest.java | 56 +--- .../streaming/util/WindowingTestHarness.java| 2 - 4 files changed, 3 insertions(+), 81 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index de316e7..c5f1ca2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -20,7 +20,6 @@ package org.apache.flink.streaming.runtime.operators.windowing; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.AppendingState; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -34,7 +33,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.core.fs.FSDataInputStream; @@ -98,7 +96,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; @Internal public class WindowOperator extends AbstractUdfStreamOperator> - implements OneInputStreamOperator, Triggerable, InputTypeConfigurable { + implements OneInputStreamOperator, Triggerable { private static final long serialVersionUID = 1L; @@ -115,12 +113,6 @@ public class WindowOperator protected final StateDescriptor, ?> windowStateDescriptor; /** -* This is used to copy the incoming element because it can be put into several window -* buffers. -*/ - protected TypeSerializer inputSerializer; - - /** * For serializing the key in checkpoints. */ protected final TypeSerializer keySerializer; @@ -211,21 +203,11 @@ public class WindowOperator } @Override - @SuppressWarnings("unchecked") - public final void setInputType(TypeInformation type, ExecutionConfig executionConfig) { - inputSerializer = (TypeSerializer) type.createSerializer(executionConfig); - } - - @Override public final void open() throws Exception { super.open(); timestampedCollector = new TimestampedCollector<>(output); - if (inputSerializer == null) { - throw new IllegalStateException("Input serializer was not set."); - } - // these could already be initialized from restoreState() if (watermarkTimers == null) { watermarkTimers = new HashSet<>(); http://git-wip-us.apache.org/repos/asf/flink/blob/1cd8d4f4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java index 681a334..8f3af15 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java +++ b/flink-streaming-java/
[09/17] flink git commit: [FLINK-4748] [streaming api] Make timers in Ingestion Time source context properly cancelable.
[FLINK-4748] [streaming api] Make timers in Ingestion Time source context properly cancelable. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/dd3416fd Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/dd3416fd Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/dd3416fd Branch: refs/heads/master Commit: dd3416fde7b934584d4e18bc54d79ed7858556c2 Parents: beb31fc Author: Stephan Ewen Authored: Tue Oct 4 22:29:28 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 19:36:14 2016 +0200 -- .../api/operators/StreamSourceContexts.java | 31 1 file changed, 19 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/dd3416fd/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java index a290deb..d0c4e15 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamSourceContexts.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.streaming.api.operators; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -115,9 +116,9 @@ public class StreamSourceContexts { private final Output> output; private final StreamRecord reuse; - private final ScheduledFuture watermarkTimer; private final long watermarkInterval; + private volatile ScheduledFuture nextWatermarkTimer; private volatile long nextWatermarkTime; private AutomaticWatermarkContext( @@ -130,13 +131,13 @@ public class StreamSourceContexts { this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpoint lock cannot be null."); this.output = Preconditions.checkNotNull(output, "The output cannot be null."); - Preconditions.checkArgument(watermarkInterval > 1L, "The watermark interval cannot be smaller than 1 ms."); + Preconditions.checkArgument(watermarkInterval >= 1L, "The watermark interval cannot be smaller than 1 ms."); this.watermarkInterval = watermarkInterval; this.reuse = new StreamRecord<>(null); long now = this.timeService.getCurrentProcessingTime(); - this.watermarkTimer = this.timeService.registerTimer(now + watermarkInterval, + this.nextWatermarkTimer = this.timeService.registerTimer(now + watermarkInterval, new WatermarkEmittingTask(this.timeService, lock, output)); } @@ -178,8 +179,9 @@ public class StreamSourceContexts { } // we can shutdown the timer now, no watermarks will be needed any more - if (watermarkTimer != null) { - watermarkTimer.cancel(true); + final ScheduledFuture nextWatermarkTimer = this.nextWatermarkTimer; + if (nextWatermarkTimer != null) { + nextWatermarkTimer.cancel(true); } } } @@ -191,8 +193,9 @@ public class StreamSourceContexts { @Override public void close() { - if (watermarkTimer != null) { - watermarkTimer.cancel(true); + final ScheduledFuture nextWatermarkTimer = this.nextWatermarkTimer; + if (nextWatermarkTimer != null) { + nextWatermarkTimer.cancel(true); } } @@ -202,10 +205,13 @@ public class StreamSourceContexts { private final Object lock; private final Output> output; - private WatermarkEmittingTask(TimeServiceProvider timeService, Object checkpointLock, Output> output) { - this.timeService = Preconditions.checkNotNull(timeService, "Time Service cannot be null."); - this.lock = Preconditions.checkNotNull(checkpointLock, "The checkpo
[17/17] flink git commit: [hotfix] [tests] Remove leftover sysout logging code
[hotfix] [tests] Remove leftover sysout logging code Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/417c5a4b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/417c5a4b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/417c5a4b Branch: refs/heads/master Commit: 417c5a4b429eb8766ff582d424a64a1cd6aa54d3 Parents: 4fc54e3 Author: Stephan Ewen Authored: Wed Oct 5 20:10:13 2016 +0200 Committer: Stephan Ewen Committed: Wed Oct 5 20:31:56 2016 +0200 -- .../streaming/runtime/operators/windowing/WindowOperatorTest.java | 3 --- 1 file changed, 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/417c5a4b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index cda6524..ba803e3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -837,9 +837,6 @@ public class WindowOperatorTest extends TestLogger { expectedOutput.add(new StreamRecord<>(new Tuple2<>("key1", 4), Long.MAX_VALUE)); expectedOutput.add(new StreamRecord<>(new Tuple2<>("key2", 4), Long.MAX_VALUE)); - System.out.println("BEFORE GOT: " + outputBeforeClose); - System.out.println("GOT: " + testHarness.getOutput()); - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, Iterables.concat(outputBeforeClose, testHarness.getOutput()), new Tuple2ResultSortComparator()); testHarness.close();
flink git commit: [hotfix] [streaming api] Re-register timers in open() instead of restore()
Repository: flink Updated Branches: refs/heads/master 6efb7c266 -> 97c71675a [hotfix] [streaming api] Re-register timers in open() instead of restore() This makes sure timers cannot fire prior to the operator being opened. This closes #2602 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/97c71675 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/97c71675 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/97c71675 Branch: refs/heads/master Commit: 97c71675ad9e2b50b818362f92faef9eb44da627 Parents: 6efb7c2 Author: kl0u Authored: Thu Oct 6 10:44:58 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 7 11:01:49 2016 +0200 -- .../runtime/operators/windowing/WindowOperator.java | 9 + 1 file changed, 5 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/97c71675/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java index c5f1ca2..4d8f655 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java @@ -230,6 +230,11 @@ public class WindowOperator if (windowAssigner instanceof MergingWindowAssigner) { mergingWindowsByKey = new HashMap<>(); } + + // re-register the restored timers (if any) + if (processingTimeTimersQueue.size() > 0) { + nextTimer = getTimerService().registerTimer(processingTimeTimersQueue.peek().timestamp, this); + } } @Override @@ -881,10 +886,6 @@ public class WindowOperator processingTimeTimersQueue.add(timer); processingTimeTimers.add(timer); } - - if (numProcessingTimeTimers > 0) { - nextTimer = getTimerService().registerTimer(processingTimeTimersQueue.peek().timestamp, this); - } } private void snapshotTimers(DataOutputView out) throws IOException {
[2/5] flink git commit: [FLINK-4786] [tests] Fix BarrierBufferTest validation of alignment time computation
[FLINK-4786] [tests] Fix BarrierBufferTest validation of alignment time computation Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/14e5ecb4 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/14e5ecb4 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/14e5ecb4 Branch: refs/heads/master Commit: 14e5ecb46c14e8ce2885ffe4ec83cdce2a13f76c Parents: 877c267 Author: Stephan Ewen Authored: Mon Oct 10 12:15:59 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:17:04 2016 +0200 -- .../org/apache/flink/streaming/runtime/io/BarrierBufferTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/14e5ecb4/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index 2cabd70..59242e8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -442,10 +442,9 @@ public class BarrierBufferTest { check(sequence[20], buffer.getNextNonBlocked()); check(sequence[23], buffer.getNextNonBlocked()); - validateAlignmentTime(startTs, buffer); - // checkpoint 2 completed check(sequence[12], buffer.getNextNonBlocked()); + validateAlignmentTime(startTs, buffer); check(sequence[25], buffer.getNextNonBlocked()); check(sequence[27], buffer.getNextNonBlocked()); check(sequence[30], buffer.getNextNonBlocked());
[4/5] flink git commit: [hotfix] [tests] Remove leftover sysout logging from AccumulatingAlignedProcessingTimeWindowOperatorTest
[hotfix] [tests] Remove leftover sysout logging from AccumulatingAlignedProcessingTimeWindowOperatorTest Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c1825e3a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c1825e3a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c1825e3a Branch: refs/heads/master Commit: c1825e3aaf27ad5270eb5b9120df6d28b54578f1 Parents: 14e5ecb Author: Stephan Ewen Authored: Mon Oct 10 12:19:37 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:19:37 2016 +0200 -- .../AccumulatingAlignedProcessingTimeWindowOperatorTest.java| 1 - 1 file changed, 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c1825e3a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java index c82392a..8a3d919 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java @@ -521,7 +521,6 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest { } // draw a snapshot and dispose the window - System.out.println("GOT: " + testHarness.getOutput()); int beforeSnapShot = testHarness.getOutput().size(); StreamStateHandle state = testHarness.snapshot(1L, System.currentTimeMillis()); List resultAtSnapshot = extractFromStreamRecords(testHarness.getOutput());
[1/5] flink git commit: [FLINK-4776] [distributed coordination] Move ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder
Repository: flink Updated Branches: refs/heads/master 5e30ba384 -> 33c36e62a [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/ed96cb53 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ed96cb53 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ed96cb53 Branch: refs/heads/master Commit: ed96cb53bfcbe96dd36248839ec4f2bffe60971b Parents: 5e30ba3 Author: Stephan Ewen Authored: Fri Oct 7 19:58:24 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:17:03 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/ed96cb53/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/ed96cb53/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/ed96cb53/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 000..1c6eb8d --- /de
[5/5] flink git commit: [hotfix] [tests] Fix PowerMock warnings concerning log4j
[hotfix] [tests] Fix PowerMock warnings concerning log4j Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/33c36e62 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/33c36e62 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/33c36e62 Branch: refs/heads/master Commit: 33c36e62a42d46f8aebe7b90ad91b529826136b6 Parents: c1825e3 Author: Stephan Ewen Authored: Mon Oct 10 12:22:38 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:22:38 2016 +0200 -- .../apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/33c36e62/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index 10fc400..21c894e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -53,7 +53,7 @@ import java.util.concurrent.atomic.AtomicLong; */ @RunWith(PowerMockRunner.class) @PrepareForTest({ResultPartitionWriter.class}) -@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"}) +@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*", "org.apache.log4j.*" }) public class SourceStreamTaskTest {
[3/5] flink git commit: [hotfix] [tests] Increase robustness of Fast Time Window Operator Tests
[hotfix] [tests] Increase robustness of Fast Time Window Operator Tests Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/877c267b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/877c267b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/877c267b Branch: refs/heads/master Commit: 877c267b8f7d7f63b07598e1536c7b42567c8a8b Parents: ed96cb5 Author: Stephan Ewen Authored: Mon Oct 10 11:19:43 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:17:04 2016 +0200 -- ...ulatingAlignedProcessingTimeWindowOperatorTest.java | 8 ...egatingAlignedProcessingTimeWindowOperatorTest.java | 13 + 2 files changed, 21 insertions(+) -- http://git-wip-us.apache.org/repos/asf/flink/blob/877c267b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java index 2f687f6..c82392a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java @@ -264,6 +264,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest { // get and verify the result out.waitForNElements(numElements, 60_000); + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } @@ -322,6 +324,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest { Thread.sleep(1); } + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } @@ -407,6 +411,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest { Collections.sort(result); assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), result); + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } @@ -463,6 +469,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest { Collections.sort(result); assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result); + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } http://git-wip-us.apache.org/repos/asf/flink/blob/877c267b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java index cd82a9c..12a842f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java @@ -277,6 +277,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest { List> result = out.getElements(); assertEquals(numElements, result.size()); + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } @@ -352,6 +354,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest { List> result = out.getElements(); + timerService.quiesceAndAwaitPending(); + synchronized (lock) { op.close(); } @@ -41
flink git commit: [FLINK-4788] [streaming api] Fix state backend classloading from configuration
Repository: flink Updated Branches: refs/heads/release-1.1 7267562bb -> d619f51ac [FLINK-4788] [streaming api] Fix state backend classloading from configuration Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d619f51a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d619f51a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d619f51a Branch: refs/heads/release-1.1 Commit: d619f51ac8f922c0cf1d1e789c5141076128f04e Parents: 7267562 Author: Stephan Ewen Authored: Mon Oct 10 14:33:57 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 14:40:52 2016 +0200 -- .../streaming/runtime/tasks/StreamTask.java | 3 +- .../streaming/runtime/tasks/StreamTaskTest.java | 47 +--- 2 files changed, 42 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/d619f51a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 940f699..d56c9bf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -684,8 +684,7 @@ public abstract class StreamTask> backendName = "jobmanager"; } - backendName = backendName.toLowerCase(); - switch (backendName) { + switch (backendName.toLowerCase()) { case "jobmanager": LOG.info("State backend is set to heap memory (checkpoint to jobmanager)"); stateBackend = MemoryStateBackend.create(); http://git-wip-us.apache.org/repos/asf/flink/blob/d619f51a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index e9d583c..83eb4bb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -22,8 +22,8 @@ import akka.actor.ActorRef; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -41,8 +41,12 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateBackendFactory; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.Output; @@ -85,7 +89,7 @@ public class StreamTaskTest { StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStreamOperator(new SlowlyDeserializingOperator()); - Task task = createTask(SourceStreamTask.class, cfg); + Task task = createTask(SourceStreamTask.class, cfg, new Configuration()); task.startTaskThread(); // wait until the task thread reached state RUNNING @@ -120,14 +124,37 @@ public class StreamTaskTest { } } + @Test + public void testStateBackendLoading() throws Exception { + Configuration taskManagerCo
[2/2] flink git commit: [FLINK-4768] [core] Migrate high-availability configuration parameters to ConfigOptions
[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/abc1657b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/abc1657b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/abc1657b Branch: refs/heads/master Commit: abc1657bac83c151a1a345220942b02fcde4653a Parents: cdebb0e Author: Stephan Ewen Authored: Sat Oct 8 01:41:02 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:37:06 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/abc1657b/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 { 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/abc1657b/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 000..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 copy
[1/2] flink git commit: [FLINK-4764] [core] Introduce Config Options
Repository: flink Updated Branches: refs/heads/master 33c36e62a -> abc1657ba [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/cdebb0ef Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cdebb0ef Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cdebb0ef Branch: refs/heads/master Commit: cdebb0ef100179e810f6e110b974632bd49c653f Parents: 33c36e6 Author: Stephan Ewen Authored: Fri Oct 7 15:24:44 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 12:34:11 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/cdebb0ef/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 000..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. + * + * {@code ConfigOptions} are built via the {@link ConfigOptions} class. + * Once created, a config option is immutable. + * + * @param The type of value associated with the configuration option. + */ +@PublicEvolving +public class ConfigOption { + + 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 defaultValueThe 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 defaultValueThe 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
flink git commit: [FLINK-4788] [streaming api] Fix state backend classloading from configuration
Repository: flink Updated Branches: refs/heads/master 1836e08f0 -> 9e17cbd6b [FLINK-4788] [streaming api] Fix state backend classloading from configuration Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9e17cbd6 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9e17cbd6 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9e17cbd6 Branch: refs/heads/master Commit: 9e17cbd6b768f73299a6a344fdf44539802fb76c Parents: 1836e08 Author: Stephan Ewen Authored: Mon Oct 10 14:33:57 2016 +0200 Committer: Stephan Ewen Committed: Mon Oct 10 20:02:33 2016 +0200 -- .../streaming/runtime/tasks/StreamTask.java | 3 +- .../streaming/runtime/tasks/StreamTaskTest.java | 95 +--- 2 files changed, 43 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9e17cbd6/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java -- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 8ada6d3..4893fed 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -752,8 +752,7 @@ public abstract class StreamTask> backendName = "jobmanager"; } - backendName = backendName.toLowerCase(); - switch (backendName) { + switch (backendName.toLowerCase()) { case "jobmanager": LOG.info("State backend is set to heap memory (checkpoint to jobmanager)"); stateBackend = new MemoryStateBackend(); http://git-wip-us.apache.org/repos/asf/flink/blob/9e17cbd6/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java -- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 47a4090..8aae19f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -18,12 +18,12 @@ package org.apache.flink.streaming.runtime.tasks; -import akka.actor.ActorRef; - import akka.dispatch.Futures; + import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -34,7 +34,6 @@ import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.filecache.FileCache; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.netty.PartitionStateChecker; @@ -44,8 +43,10 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; +import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateBackendFactory; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -60,10 +61,10 @@ import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedValue; + import org.junit.Test; import scala.concurrent.Await; -import scala.conc
[3/8] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods
[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 Authored: Wed Oct 12 23:25:16 2016 +0200 Committer: Stephan Ewen 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 b
[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 {
[5/8] flink git commit: [FLINK-4735] [cluster management] Implements some job execution related RPC calls on the JobManager
[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 Authored: Tue Oct 4 23:00:22 2016 +0800 Committer: Stephan Ewen 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.Se
[7/8] flink git commit: [FLINK-4764] [core] Introduce Config Options
[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 Authored: Fri Oct 7 15:24:44 2016 +0200 Committer: Stephan Ewen 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 000..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. + * + * {@code ConfigOptions} are built via the {@link ConfigOptions} class. + * Once created, a config option is immutable. + * + * @param The type of value associated with the configuration option. + */ +@PublicEvolving +public class ConfigOption { + + 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 defaultValueThe 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 defaultValueThe 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; + } + + // --
[2/8] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods
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.CheckpointS
[6/8] flink git commit: [FLINK-4776] [distributed coordination] Move ExecutionGraph initialization into the dedicated class ExecutionGraphBuilder
[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 Authored: Fri Oct 7 19:58:24 2016 +0200 Committer: Stephan Ewen 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 000..1c6eb8d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph
[4/8] flink git commit: [FLINK-4768] [core] Migrate high-availability configuration parameters to ConfigOptions
[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 Authored: Sat Oct 8 01:41:02 2016 +0200 Committer: Stephan Ewen 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 { 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 000..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 copy
[8/8] flink git commit: [FLINK-4339] [cluster management] Implement Slot Pool core on JobManager side
[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 Authored: Thu Oct 13 04:59:46 2016 +0800 Committer: Stephan Ewen 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; + } +
flink git commit: [FLINK-4373] [cluster management] Introduce AllocationID, ResourceProfile, and AllocatedSlot
Repository: flink Updated Branches: refs/heads/master 5783671c2 -> f6d866817 [FLINK-4373] [cluster management] Introduce AllocationID, ResourceProfile, and AllocatedSlot These classes are introduced as part of the cluster management rework. This closes #2630 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f6d86681 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f6d86681 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f6d86681 Branch: refs/heads/master Commit: f6d8668175fb94f338037fd1ab40d2a2e344d097 Parents: 5783671 Author: Stephan Ewen Authored: Thu Oct 13 18:50:18 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 11:33:35 2016 +0200 -- .../clusterframework/types/AllocationID.java| 50 ++ .../clusterframework/types/ResourceProfile.java | 122 +++ .../flink/runtime/instance/SharedSlot.java | 53 ++- .../flink/runtime/instance/SimpleSlot.java | 57 ++- .../org/apache/flink/runtime/instance/Slot.java | 77 +++--- .../runtime/jobmanager/slots/AllocatedSlot.java | 152 +++ 6 files changed, 488 insertions(+), 23 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/f6d86681/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java new file mode 100644 index 000..59d8f9d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java @@ -0,0 +1,50 @@ +/* + * 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.clusterframework.types; + +import org.apache.flink.util.AbstractID; + +/** + * Unique identifier for a slot allocated by a JobManager from a TaskManager. + * Also identifies a pending allocation request, and is constant across retries. + * + * This ID is used for all synchronization of the status of Slots from TaskManagers + * that are not free (i.e., have been allocated by a job). + */ +public class AllocationID extends AbstractID { + + private static final long serialVersionUID = 1L; + + /** +* Constructs a new random AllocationID. +*/ + public AllocationID() { + super(); + } + + /** +* Constructs a new AllocationID with the given parts. +* +* @param lowerPart the lower bytes of the ID +* @param upperPart the higher bytes of the ID +*/ + public AllocationID(long lowerPart, long upperPart) { + super(lowerPart, upperPart); + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/f6d86681/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 new file mode 100644 index 000..7a25de1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java @@ -0,0 +1,122 @@ +/* + * 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 Licens
[07/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
[FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package The TaskExecutor, the JobMaster and the ResourceManager were still contained in the rpc package. With this commit, they will be moved out of this package. Now they are contained in dedicated packages on the o.a.f.runtime level. This closes #2438. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9c247d1f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9c247d1f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9c247d1f Branch: refs/heads/flip-6 Commit: 9c247d1ffd732d101145ef3ef6e8050151128f5c Parents: 4b077af Author: Till Rohrmann Authored: Mon Aug 29 16:35:29 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:39 2016 +0200 -- .../runtime/clusterframework/SlotManager.java | 525 .../flink/runtime/jobmaster/JobMaster.java | 244 ++ .../runtime/jobmaster/JobMasterGateway.java | 45 + .../registration/RegistrationResponse.java | 84 ++ .../registration/RetryingRegistration.java | 296 +++ .../resourcemanager/JobMasterRegistration.java | 35 + .../resourcemanager/RegistrationResponse.java | 43 + .../resourcemanager/ResourceManager.java| 214 + .../resourcemanager/ResourceManagerGateway.java | 77 ++ .../runtime/resourcemanager/SlotAssignment.java | 25 + .../runtime/resourcemanager/SlotManager.java| 523 .../runtime/resourcemanager/SlotRequest.java| 74 ++ .../flink/runtime/rpc/jobmaster/JobMaster.java | 244 -- .../runtime/rpc/jobmaster/JobMasterGateway.java | 45 - .../rpc/registration/RegistrationResponse.java | 84 -- .../rpc/registration/RetryingRegistration.java | 296 --- .../resourcemanager/JobMasterRegistration.java | 35 - .../resourcemanager/RegistrationResponse.java | 43 - .../rpc/resourcemanager/ResourceManager.java| 214 - .../resourcemanager/ResourceManagerGateway.java | 77 -- .../rpc/resourcemanager/SlotAssignment.java | 25 - .../rpc/resourcemanager/SlotRequest.java| 74 -- .../runtime/rpc/taskexecutor/SlotReport.java| 56 -- .../runtime/rpc/taskexecutor/SlotStatus.java| 129 --- .../runtime/rpc/taskexecutor/TaskExecutor.java | 827 --- .../taskexecutor/TaskExecutorConfiguration.java | 151 .../rpc/taskexecutor/TaskExecutorGateway.java | 35 - .../TaskExecutorRegistrationSuccess.java| 75 -- ...TaskExecutorToResourceManagerConnection.java | 198 - .../flink/runtime/taskexecutor/SlotReport.java | 56 ++ .../flink/runtime/taskexecutor/SlotStatus.java | 129 +++ .../runtime/taskexecutor/TaskExecutor.java | 827 +++ .../taskexecutor/TaskExecutorConfiguration.java | 151 .../taskexecutor/TaskExecutorGateway.java | 35 + .../TaskExecutorRegistrationSuccess.java| 75 ++ ...TaskExecutorToResourceManagerConnection.java | 198 + .../clusterframework/ClusterShutdownITCase.java | 156 .../clusterframework/ResourceManagerITCase.java | 162 .../clusterframework/ResourceManagerTest.java | 338 .../clusterframework/SlotManagerTest.java | 540 .../registration/RetryingRegistrationTest.java | 336 .../registration/TestRegistrationGateway.java | 85 ++ .../resourcemanager/ClusterShutdownITCase.java | 156 .../resourcemanager/ResourceManagerHATest.java | 76 ++ .../resourcemanager/ResourceManagerITCase.java | 162 .../resourcemanager/ResourceManagerTest.java| 338 .../resourcemanager/SlotManagerTest.java| 538 .../runtime/rpc/akka/AkkaRpcServiceTest.java| 14 - .../registration/RetryingRegistrationTest.java | 336 .../registration/TestRegistrationGateway.java | 85 -- .../resourcemanager/ResourceManagerHATest.java | 76 -- .../rpc/taskexecutor/TaskExecutorTest.java | 117 --- .../runtime/taskexecutor/TaskExecutorTest.java | 117 +++ 53 files changed, 4939 insertions(+), 4957 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java deleted file mode 100644 index cc140a1..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java +++ /dev/null @@ -1,525 +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
[22/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Implement TaskManager component's startup
[FLINK-4505] [cluster mngt] Implement TaskManager component's startup The TaskManagerRunner now contains the startup logic for the TaskManager's components. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c34f13c2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c34f13c2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c34f13c2 Branch: refs/heads/flip-6 Commit: c34f13c26bf215fbde6c3f28547f1a31ca6cbc99 Parents: b8c6b99 Author: 淘江 Authored: Fri Sep 2 18:00:49 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../runtime/taskexecutor/TaskExecutor.java | 766 +-- .../runtime/taskmanager/TaskManagerRunner.java | 749 ++ .../runtime/taskexecutor/TaskExecutorTest.java | 53 +- 3 files changed, 804 insertions(+), 764 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c34f13c2/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 9d9ad2a..8ce2780 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,74 +18,29 @@ package org.apache.flink.runtime.taskexecutor; -import akka.actor.ActorSystem; -import com.typesafe.config.Config; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.io.network.ConnectionManager; -import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; -import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; -import org.apache.flink.runtime.query.netty.KvStateServer; import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.Preconditions; import org.jboss.netty.channel.ChannelException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.memory.HeapMemorySegment; -import org.apache.flink.core.memory.HybridMemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; 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.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.taskmanager.MemoryLogger; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; -import org.apache.flink.util.MathUtils; -import org.apache.flink.util.NetUtils; - -import scala.Tuple2; -import scala.Option; -import scala.Some; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.F
[03/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java deleted file mode 100644 index 2ee280f..000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java +++ /dev/null @@ -1,540 +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.clusterframework; - -import org.apache.flink.api.common.JobID; -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.clusterframework.types.ResourceSlot; -import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; -import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; -import org.junit.Before; -import org.junit.Test; - -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; - -public class SlotManagerTest { - - private static final double DEFAULT_TESTING_CPU_CORES = 1.0; - - private static final long DEFAULT_TESTING_MEMORY = 512; - - private static final ResourceProfile DEFAULT_TESTING_PROFILE = - new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY); - - private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = - new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2); - - private ResourceManagerGateway resourceManagerGateway; - - @Before - public void setUp() { - resourceManagerGateway = mock(ResourceManagerGateway.class); - } - - /** -* Tests that there are no free slots when we request, need to allocate from cluster manager master -*/ - @Test - public void testRequestSlotWithoutFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); - - assertEquals(0, slotManager.getAllocatedSlotCount()); - assertEquals(0, slotManager.getFreeSlotCount()); - assertEquals(1, slotManager.getPendingRequestCount()); - assertEquals(1, slotManager.getAllocatedContainers().size()); - assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0)); - } - - /** -* Tests that there are some free slots when we request, and the request is fulfilled immediately -*/ - @Test - public void testRequestSlotWithFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - - directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); - assertEquals(1, slotManager.getFreeSlotCount()); - - slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); - assertEquals(1, slotManager.getAllocatedSlotCount()); - assertEquals(0, slotManager.getFreeSlotCount()); - assertEquals(0, slotManager.getPendingRequestCount()); - assertEquals(0, slotManager.getAllocatedContainers().size()); - } - - /** -* Tests that there are some free slots when we request, but none of them are suitable -*/ - @Test - public void testRequestSlotWithoutSuitableSlot() { - TestingSlotManager slotMan
[08/50] [abbrv] flink git commit: [FLINK-4516] leader election of resourcemanager
[FLINK-4516] leader election of resourcemanager - add serial rpc service - add a special rpcService implementation which directly executes the asynchronous calls serially one by one, it is just for testcase - Change ResourceManagerLeaderContender code and TestingSerialRpcService code - override shutdown logic to stop leadershipService - use a mocked RpcService rather than TestingSerialRpcService for resourceManager HA test This closes #2427 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4b077af4 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4b077af4 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4b077af4 Branch: refs/heads/flip-6 Commit: 4b077af4dfab4e9c6e1ae4e6f4865f1f319deb0c Parents: 6e22c64 Author: beyond1920 Authored: Sat Aug 27 14:14:28 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:39 2016 +0200 -- .../HighAvailabilityServices.java | 7 + .../runtime/highavailability/NonHaServices.java | 5 + .../rpc/resourcemanager/ResourceManager.java| 111 +- .../TestingHighAvailabilityServices.java| 19 +- .../runtime/rpc/TestingSerialRpcService.java| 369 +++ .../resourcemanager/ResourceManagerHATest.java | 76 6 files changed, 578 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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 73e4f1f..298147c 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 @@ -40,6 +40,13 @@ public interface HighAvailabilityServices { LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; /** +* Gets the leader election service for the cluster's resource manager. +* @return +* @throws Exception +*/ + LeaderElectionService getResourceManagerLeaderElectionService() throws Exception; + + /** * Gets the leader election service for the given job. * * @param jobID The identifier of the job running the election. http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/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 3d2769b..292a404 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 @@ -61,6 +61,11 @@ public class NonHaServices implements HighAvailabilityServices { } @Override + public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception { + return new StandaloneLeaderElectionService(); + } + + @Override public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception { return new StandaloneLeaderElectionService(); } http://git-wip-us.apache.org/repos/asf/flink/blob/4b077af4/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java index 6f34465..f7147c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java @@ -20,24 +20,26 @@ package org.apache.flink.runtime.rpc.resourcemanager; import akka.dispatch.Mapper; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.
[15/50] [abbrv] flink git commit: [hotfix] Remove unused imports from SlotRequestRegistered/Rejected and ResourceSlot
[hotfix] Remove unused imports from SlotRequestRegistered/Rejected and ResourceSlot Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b2c5c0fd Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b2c5c0fd Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b2c5c0fd Branch: refs/heads/flip-6 Commit: b2c5c0fd154b3233ef1fb5a65175a94f3111dcd3 Parents: 74570d4 Author: Till Rohrmann Authored: Wed Sep 21 11:47:53 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:40 2016 +0200 -- .../flink/runtime/clusterframework/types/ResourceSlot.java | 4 .../flink/runtime/resourcemanager/SlotRequestRegistered.java | 4 ++-- .../flink/runtime/resourcemanager/SlotRequestRejected.java | 4 ++-- 3 files changed, 4 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java index 5fb8aee..4a91a79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -20,8 +20,6 @@ package org.apache.flink.runtime.clusterframework.types; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import java.io.Serializable; - import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -30,8 +28,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull; */ public class ResourceSlot implements ResourceIDRetrievable { - private static final long serialVersionUID = -5853720153136840674L; - /** The unique identification of this slot */ private final SlotID slotId; http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java index 6b7f6dc..f719dce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRegistered.java @@ -20,13 +20,13 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import java.io.Serializable; - /** * Acknowledgment by the ResourceManager for a SlotRequest from the JobManager */ public class SlotRequestRegistered extends SlotRequestReply { + private static final long serialVersionUID = 4760320859275256855L; + public SlotRequestRegistered(AllocationID allocationID) { super(allocationID); } http://git-wip-us.apache.org/repos/asf/flink/blob/b2c5c0fd/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java index cb3ec72..282a7d5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequestRejected.java @@ -20,13 +20,13 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import java.io.Serializable; - /** * Rejection message by the ResourceManager for a SlotRequest from the JobManager */ public class SlotRequestRejected extends SlotRequestReply { + private static final long serialVersionUID = 9049346740895325144L; + public SlotRequestRejected(AllocationID allocationID) { super(allocationID); }
[06/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java new file mode 100644 index 000..896421b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.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.resourcemanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager. + */ +public class SlotRequest implements Serializable { + + private static final long serialVersionUID = -6586877187990445986L; + + /** The JobID of the slot requested for */ + private final JobID jobId; + + /** The unique identification of this request */ + private final AllocationID allocationId; + + /** The resource profile of the required slot */ + private final ResourceProfile resourceProfile; + + public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) { + this.jobId = checkNotNull(jobId); + this.allocationId = checkNotNull(allocationId); + this.resourceProfile = checkNotNull(resourceProfile); + } + + /** +* Get the JobID of the slot requested for. +* @return The job id +*/ + public JobID getJobId() { + return jobId; + } + + /** +* Get the unique identification of this request +* @return the allocation id +*/ + public AllocationID getAllocationId() { + return allocationId; + } + + /** +* Get the resource profile of the desired slot +* @return The resource profile +*/ + public ResourceProfile getResourceProfile() { + return resourceProfile; + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java deleted file mode 100644 index a046cb8..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java +++ /dev/null @@ -1,244 +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.rpc.jobmaster; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderele
[23/50] [abbrv] flink git commit: [FLINK-4535] rebase and refine
[FLINK-4535] rebase and refine Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/485ef003 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/485ef003 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/485ef003 Branch: refs/heads/flip-6 Commit: 485ef0035fe3f0d4335d880868ab9beb18731fdf Parents: c9764c8 Author: Maximilian Michels Authored: Wed Sep 21 20:20:25 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../resourcemanager/JobMasterRegistration.java | 64 .../resourcemanager/ResourceManager.java| 322 --- .../resourcemanager/ResourceManagerGateway.java | 36 +-- .../TaskExecutorRegistration.java | 2 +- .../slotmanager/SlotManager.java| 1 - .../ResourceManagerJobMasterTest.java | 174 ++ .../ResourceManagerTaskExecutorTest.java| 135 .../resourcemanager/ResourceManagerTest.java| 141 .../slotmanager/SlotProtocolTest.java | 43 ++- 9 files changed, 574 insertions(+), 344 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java deleted file mode 100644 index 981441f..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ /dev/null @@ -1,64 +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.resourcemanager; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; - -import java.util.UUID; - -/** - * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master - */ -public class JobMasterRegistration implements LeaderRetrievalListener { - - private final JobMasterGateway gateway; - private final JobID jobID; - private final UUID leaderSessionID; - private LeaderRetrievalListener retriever; - - public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) { - this.gateway = gateway; - this.jobID = jobID; - this.leaderSessionID = leaderSessionID; - } - - public JobMasterGateway getGateway() { - return gateway; - } - - public UUID getLeaderSessionID() { - return leaderSessionID; - } - - public JobID getJobID() { - return jobID; - } - - @Override - public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { - - } - - @Override - public void handleError(Exception exception) { - - } -} http://git-wip-us.apache.org/repos/asf/flink/blob/485ef003/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 15692b6..88b8a11 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,29 +18,41 @@ package org.apache.flink.runtime.resourcemanager; -import akka.dispatch.Futures; -import akka.dispatch.Mapper; - import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; import org.apache.flink.runti
[50/50] [abbrv] flink git commit: [FLINK-4339] [cluster management] Implement Slot Pool core on JobManager side
[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/7aca811d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7aca811d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7aca811d Branch: refs/heads/flip-6 Commit: 7aca811df96ee0628fc4d274971b3ffc6d4b6eb7 Parents: 0615b62 Author: Kurt Young Authored: Thu Oct 13 04:59:46 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:32:27 2016 +0200 -- .../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 7 files changed, 1403 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/7aca811d/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/7aca811d/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 000..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 p
[12/50] [abbrv] flink git commit: [hotfix] Add methods defined in the gateway to the ResourceManager and TaskExecutor
[hotfix] Add methods defined in the gateway to the ResourceManager and TaskExecutor Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/fbd38671 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/fbd38671 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/fbd38671 Branch: refs/heads/flip-6 Commit: fbd38671f33d2579cf2856c7f9b50ffc0335113f Parents: b2c5c0f Author: Till Rohrmann Authored: Wed Sep 21 14:14:05 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:40 2016 +0200 -- .../runtime/resourcemanager/ResourceManager.java | 1 + .../resourcemanager/ResourceManagerGateway.java | 2 +- .../flink/runtime/taskexecutor/TaskExecutor.java | 15 +++ .../runtime/taskexecutor/TaskExecutorGateway.java| 6 +++--- 4 files changed, 20 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 29aba1a..d9a7134 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.resourcemanager; +import akka.dispatch.Futures; import akka.dispatch.Mapper; import org.apache.flink.annotation.VisibleForTesting; http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index e5c8b64..c8e3488 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -58,7 +58,7 @@ public interface ResourceManagerGateway extends RpcGateway { * @param slotRequest Slot request * @return Future slot assignment */ - Future requestSlot(SlotRequest slotRequest); + Future requestSlot(SlotRequest slotRequest); /** * http://git-wip-us.apache.org/repos/asf/flink/blob/fbd38671/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 a455fe2..fadae5f 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 @@ -21,6 +21,7 @@ package org.apache.flink.runtime.taskexecutor; import akka.actor.ActorSystem; import akka.util.Timeout; import com.typesafe.config.Config; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.LocalConnectionManager; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -30,6 +31,8 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; import org.apache.flink.runtime.query.netty.KvStateServer; +import org.apache.flink.runtime.resourcemanager.SlotRequestRegistered; +import org.apache.flink.runtime.resourcemanager.SlotRequestReply; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -201,6 +204,18 @@ public class TaskExecutor extends RpcEndpoint { } /** +* Requests a slot from the TaskManager +* +* @param allocationID id for the request +* @param resourceManagerLeaderID current leader id of the ResourceManager +* @return answer to the slot request +*/ + @RpcMethod + public SlotRequestReply requestSlot(AllocationID allocationID, UUID resourceManagerLeaderID) { + return new SlotRequestRegistered(allocationID); + } + +
[16/50] [abbrv] flink git commit: [FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol
http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java deleted file mode 100644 index 52d9d06..000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java +++ /dev/null @@ -1,538 +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.resourcemanager; - -import org.apache.flink.api.common.JobID; -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.clusterframework.types.ResourceSlot; -import org.apache.flink.runtime.clusterframework.types.SlotID; -import org.apache.flink.runtime.taskexecutor.SlotStatus; -import org.junit.Before; -import org.junit.Test; - -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; - -public class SlotManagerTest { - - private static final double DEFAULT_TESTING_CPU_CORES = 1.0; - - private static final long DEFAULT_TESTING_MEMORY = 512; - - private static final ResourceProfile DEFAULT_TESTING_PROFILE = - new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY); - - private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = - new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2); - - private ResourceManagerGateway resourceManagerGateway; - - @Before - public void setUp() { - resourceManagerGateway = mock(ResourceManagerGateway.class); - } - - /** -* Tests that there are no free slots when we request, need to allocate from cluster manager master -*/ - @Test - public void testRequestSlotWithoutFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); - - assertEquals(0, slotManager.getAllocatedSlotCount()); - assertEquals(0, slotManager.getFreeSlotCount()); - assertEquals(1, slotManager.getPendingRequestCount()); - assertEquals(1, slotManager.getAllocatedContainers().size()); - assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0)); - } - - /** -* Tests that there are some free slots when we request, and the request is fulfilled immediately -*/ - @Test - public void testRequestSlotWithFreeSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - - directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); - assertEquals(1, slotManager.getFreeSlotCount()); - - slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); - assertEquals(1, slotManager.getAllocatedSlotCount()); - assertEquals(0, slotManager.getFreeSlotCount()); - assertEquals(0, slotManager.getPendingRequestCount()); - assertEquals(0, slotManager.getAllocatedContainers().size()); - } - - /** -* Tests that there are some free slots when we request, but none of them are suitable -*/ - @Test - public void testRequestSlotWithoutSuitableSlot() { - TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); - - directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2); -
[26/50] [abbrv] flink git commit: [hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable.
[hotfix] [rpc] Add RpcConnectionTest to validate that connection buildup fails fast when endpoint is unreachable. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/0e495b7b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0e495b7b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0e495b7b Branch: refs/heads/flip-6 Commit: 0e495b7b29c978b02957ab149e9ae0ba9c7d634f Parents: c6954a9 Author: Stephan Ewen Authored: Wed Sep 21 13:03:17 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../flink/runtime/rpc/AsyncCallsTest.java | 4 +- .../flink/runtime/rpc/RpcCompletenessTest.java | 14 ++-- .../flink/runtime/rpc/RpcConnectionTest.java| 86 3 files changed, 96 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index e8255d4..7affdb9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -43,9 +43,9 @@ public class AsyncCallsTest extends TestLogger { // shared test members // - private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + private static final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); - private static AkkaRpcService akkaRpcService = + private static final AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, Time.milliseconds(1L)); @AfterClass http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index ee3f784..53355e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -30,6 +30,7 @@ import org.reflections.Reflections; import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,7 +70,8 @@ public class RpcCompletenessTest extends TestLogger { @SuppressWarnings("rawtypes") private void checkCompleteness(Class rpcEndpoint, Class rpcGateway) { - Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]); + List rpcMethodsFromGateway = getRpcMethodsFromGateway(rpcGateway); + Method[] gatewayMethods = rpcMethodsFromGateway.toArray(new Method[rpcMethodsFromGateway.size()]); Method[] serverMethods = rpcEndpoint.getMethods(); Map> rpcMethods = new HashMap<>(); @@ -360,13 +362,13 @@ public class RpcCompletenessTest extends TestLogger { } // Get all methods declared in current interface - for(Method method : interfaceClass.getDeclaredMethods()) { - allMethods.add(method); - } + Collections.addAll(allMethods, interfaceClass.getDeclaredMethods()); // Get all method inherited from super interface - for(Class superClass : interfaceClass.getInterfaces()) { - allMethods.addAll(getRpcMethodsFromGateway(superClass)); + for (Class superClass : interfaceClass.getInterfaces()) { + @SuppressWarnings("unchecked") + Class gatewayClass = (Class) superClass; + allMethods.addAll(getRpcMethodsFromGateway(gatewayClass)); } return allMethods; } http://git-wip-us.apache.org/repos/asf/flink/blob/0e495b7b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java new file mode 100644
[28/50] [abbrv] flink git commit: [FLINK-4687] [rpc] Add getAddress to RpcService
[FLINK-4687] [rpc] Add getAddress to RpcService This closes #2551. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/07512e06 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/07512e06 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/07512e06 Branch: refs/heads/flip-6 Commit: 07512e06acfc4bb3c48f1286ce52478c64ffb259 Parents: 59d9e67 Author: Till Rohrmann Authored: Mon Sep 26 18:01:47 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../org/apache/flink/runtime/rpc/RpcService.java| 8 .../flink/runtime/rpc/akka/AkkaRpcService.java | 16 .../apache/flink/runtime/rpc/TestingRpcService.java | 5 +++-- .../flink/runtime/rpc/TestingSerialRpcService.java | 6 ++ .../flink/runtime/rpc/akka/AkkaRpcActorTest.java| 8 .../flink/runtime/rpc/akka/AkkaRpcServiceTest.java | 5 + 6 files changed, 42 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/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 437e08b..96844ed 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,6 +33,14 @@ 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 Address of the rpc service or empty string if local rpc service +*/ + String getAddress(); + + /** * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can * be used to communicate with the rpc server. If the connection failed, then the returned * future is failed with a {@link RpcConnectionException}. http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index cee19c4..6825557 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -22,6 +22,7 @@ import akka.actor.ActorIdentity; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.actor.ActorSystem; +import akka.actor.Address; import akka.actor.Identify; import akka.actor.PoisonPill; import akka.actor.Props; @@ -75,6 +76,8 @@ public class AkkaRpcService implements RpcService { private final Set actors = new HashSet<>(4); private final long maximumFramesize; + private final String address; + private volatile boolean stopped; public AkkaRpcService(final ActorSystem actorSystem, final Time timeout) { @@ -87,6 +90,19 @@ public class AkkaRpcService implements RpcService { // only local communication maximumFramesize = Long.MAX_VALUE; } + + Address actorSystemAddress = AkkaUtils.getAddress(actorSystem); + + if (actorSystemAddress.host().isDefined()) { + address = actorSystemAddress.host().get(); + } else { + address = ""; + } + } + + @Override + public String getAddress() { + return address; } // this method does not mutate state and is thus thread-safe http://git-wip-us.apache.org/repos/asf/flink/blob/07512e06/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index f164056..47c9e24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.concurrent.impl.FlinkCompletabl
[29/50] [abbrv] flink git commit: [FLINK-4580] [rpc] Report rpc invocation exceptions to the caller
[FLINK-4580] [rpc] Report rpc invocation exceptions to the caller This closes #2526. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/59d9e67b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/59d9e67b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/59d9e67b Branch: refs/heads/flip-6 Commit: 59d9e67b5811da2a864f7784a685e3c829f4f039 Parents: 360eaf8 Author: Till Rohrmann Authored: Wed Sep 21 15:18:27 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../flink/runtime/rpc/akka/AkkaRpcActor.java| 53 .../runtime/rpc/akka/AkkaRpcActorTest.java | 34 + 2 files changed, 66 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/59d9e67b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index 59daa46..1b456a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.akka.messages.Processing; import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,11 +87,11 @@ class AkkaRpcActor> extends Untyp unstashAll(); getContext().become(new Procedure() { @Override - public void apply(Object message) throws Exception { - if (message.equals(Processing.STOP)) { + public void apply(Object msg) throws Exception { + if (msg.equals(Processing.STOP)) { getContext().unbecome(); } else { - handleMessage(message); + handleMessage(msg); } } }); @@ -130,21 +131,36 @@ class AkkaRpcActor> extends Untyp * @param rpcInvocation Rpc invocation message */ private void handleRpcInvocation(RpcInvocation rpcInvocation) { + Method rpcMethod = null; + try { String methodName = rpcInvocation.getMethodName(); Class[] parameterTypes = rpcInvocation.getParameterTypes(); - Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); + rpcMethod = lookupRpcMethod(methodName, parameterTypes); + } catch(ClassNotFoundException e) { + LOG.error("Could not load method arguments.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not load method arguments.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } catch (IOException e) { + LOG.error("Could not deserialize rpc invocation message.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not deserialize rpc invocation message.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } catch (final NoSuchMethodException e) { + LOG.error("Could not find rpc method for rpc invocation.", e); + + RpcConnectionException rpcException = new RpcConnectionException("Could not find rpc method for rpc invocation.", e); + getSender().tell(new Status.Failure(rpcException), getSelf()); + } - if (rpcMethod.getReturnType().equals(Void.TYPE)) { - // No return value to send back - try { + if (rpcMethod != null) { + try { + if (rpcMethod.getReturnType().equals(Void.TYPE)) { + // No return value to send back rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); - } catch (Throwable e) { -
[45/50] [abbrv] flink git commit: [FLINK-4746] Make TaskManagerRuntimeInfo an interface
[FLINK-4746] Make TaskManagerRuntimeInfo an interface Let the TaskManagerConfiguration implement the TaskManagerRuntimeInformation to make some of the TaskManager's configuration values accessible from different components. This closes #2599. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b3806349 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b3806349 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b3806349 Branch: refs/heads/flip-6 Commit: b380634965acf1327ac0f6ca0a64e7d5522dbea7 Parents: a00619a Author: Till Rohrmann Authored: Wed Oct 5 14:47:24 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:43 2016 +0200 -- .../runtime/taskexecutor/TaskExecutor.java | 11 +--- .../taskexecutor/TaskManagerConfiguration.java | 22 +++ .../taskmanager/TaskManagerRuntimeInfo.java | 61 ++-- .../flink/runtime/taskmanager/TaskManager.scala | 11 +--- .../operators/drivers/TestTaskContext.java | 4 +- .../testutils/BinaryOperatorTestBase.java | 4 +- .../operators/testutils/DriverTestBase.java | 4 +- .../operators/testutils/MockEnvironment.java| 8 +-- .../testutils/UnaryOperatorTestBase.java| 4 +- .../runtime/taskexecutor/TaskExecutorTest.java | 8 +-- .../runtime/taskmanager/TaskAsyncCallTest.java | 4 +- .../flink/runtime/taskmanager/TaskTest.java | 3 +- .../util/TestingTaskManagerRuntimeInfo.java | 52 + .../tasks/InterruptSensitiveRestoreTest.java| 5 +- .../runtime/tasks/StreamMockEnvironment.java| 4 +- .../streaming/runtime/tasks/StreamTaskTest.java | 4 +- 16 files changed, 98 insertions(+), 111 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/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 35b639b..a2716e5 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,7 +18,6 @@ package org.apache.flink.runtime.taskexecutor; -import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -71,7 +70,6 @@ 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.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.util.Preconditions; import java.util.HashSet; @@ -127,9 +125,6 @@ public class TaskExecutor extends RpcEndpoint { private final FileCache fileCache; - // TODO: Try to get rid of it - private final TaskManagerRuntimeInfo taskManagerRuntimeInfo; - // - resource manager private TaskExecutorToResourceManagerConnection resourceManagerConnection; @@ -177,10 +172,6 @@ public class TaskExecutor extends RpcEndpoint { this.taskManagerMetricGroup = checkNotNull(taskManagerMetricGroup); this.broadcastVariableManager = checkNotNull(broadcastVariableManager); this.fileCache = checkNotNull(fileCache); - this.taskManagerRuntimeInfo = new TaskManagerRuntimeInfo( - taskManagerLocation.getHostname(), - new UnmodifiableConfiguration(taskManagerConfiguration.getConfiguration()), - taskManagerConfiguration.getTmpDirPaths()); this.jobManagerConnections = new HashMap<>(4); @@ -308,7 +299,7 @@ public class TaskExecutor extends RpcEndpoint { checkpointResponder, libraryCache, fileCache, - taskManagerRuntimeInfo, + taskManagerConfiguration, taskMetricGroup, resultPartitionConsumableNotifier, partitionStateChecker, http://git-wip-us.apache.org/repos/asf/flink/blob/b3806349/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskM
[14/50] [abbrv] flink git commit: [FLINK-4537] rebase and refine
[FLINK-4537] rebase and refine Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/34a6854b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/34a6854b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/34a6854b Branch: refs/heads/flip-6 Commit: 34a6854b44db8e949a02700deff18475d023b7bc Parents: efc7de5 Author: Maximilian Michels Authored: Wed Sep 21 14:13:12 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:40 2016 +0200 -- .../resourcemanager/JobMasterRegistration.java | 52 +++--- .../resourcemanager/ResourceManager.java| 165 --- .../slotmanager/SlotManager.java| 29 +++- 3 files changed, 110 insertions(+), 136 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/34a6854b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java index 7b8ec70..981441f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java @@ -18,59 +18,47 @@ package org.apache.flink.runtime.resourcemanager; -<<< HEAD import org.apache.flink.api.common.JobID; -=== import org.apache.flink.runtime.jobmaster.JobMasterGateway; ->>> db98efb... rsourceManager registration with JobManager +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import java.io.Serializable; import java.util.UUID; -import static org.apache.flink.util.Preconditions.checkNotNull; - /** * This class is responsible for group the JobMasterGateway and the LeaderSessionID of a registered job master */ -public class JobMasterRegistration implements Serializable { +public class JobMasterRegistration implements LeaderRetrievalListener { -<<< HEAD - private final String address; + private final JobMasterGateway gateway; private final JobID jobID; + private final UUID leaderSessionID; + private LeaderRetrievalListener retriever; - public JobMasterRegistration(String address, JobID jobID) { - this.address = address; + public JobMasterRegistration(JobMasterGateway gateway, JobID jobID, UUID leaderSessionID) { + this.gateway = gateway; this.jobID = jobID; -=== - private static final long serialVersionUID = -2316627821716999527L; - - private final JobMasterGateway jobMasterGateway; - - private UUID jobMasterLeaderSessionID; - - public JobMasterRegistration(JobMasterGateway jobMasterGateway) { - this.jobMasterGateway = checkNotNull(jobMasterGateway); + this.leaderSessionID = leaderSessionID; } - public JobMasterRegistration(JobMasterGateway jobMasterGateway, UUID jobMasterLeaderSessionID) { - this.jobMasterGateway = checkNotNull(jobMasterGateway); - this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; + public JobMasterGateway getGateway() { + return gateway; } - public JobMasterGateway getJobMasterGateway() { - return jobMasterGateway; + public UUID getLeaderSessionID() { + return leaderSessionID; } - public void setJobMasterLeaderSessionID(UUID leaderSessionID) { - this.jobMasterLeaderSessionID = jobMasterLeaderSessionID; ->>> db98efb... rsourceManager registration with JobManager + public JobID getJobID() { + return jobID; } - public UUID getJobMasterLeaderSessionID() { - return jobMasterLeaderSessionID; + @Override + public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) { + } - public JobID getJobID() { - return jobID; + @Override + public void handleError(Exception exception) { + } } http://git-wip-us.apache.org/repos/asf/flink/blob/34a6854b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 8be1455..aae4874 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b
[39/50] [abbrv] flink git commit: [FLINK-4347][FLINK-4348] simplify SlotManager and integrate it with ResourceManager
[FLINK-4347][FLINK-4348] simplify SlotManager and integrate it with ResourceManager Instead of relying on a full synchronization of all slots information on every heartbeat, the SlotManager is now responsible for updating its state. It initially syncs all slots upon registration of the TaskExecutor. After that, it only receives notifications from the TaskExecutor when slots become available again. This simplifies the logic of the SlotManager and makes the slot allocation more predictable in case of message loss. Additional changes: - Move the slot registration and allocation report to the registration of the TaskExecutor - Let the TaskExecutor immediately notify the ResourceManager once a slot becomes free. The ResourceManager has to confirm this notification. Otherwise, the slot will be blocked because the ResourceManager's state is not in sync. - Integrate with handleSlotRequestFailedAtTaskManager and introduce fencing to protect against TaskExecutors which are not registered anymore. - introduce RPC call to notify ResourceManager about free slots - ignore out-of-date slot requests from ResourceManager at TaskExecutor - let the ResourceManager update its state instead of relying on heartbeats - provide ResourceManagerServices to SlotManager - introduce factory for SlotManager - keep task gateways and worker information in ResourceManager and inform SlotManager - add TaskExecutor test to ensure that a free slot which hasn't been confirmed by the task executor is correctly blacklisted as long as the ResourceManager has not confirmed the allocation removal. - adapt tests - update javadocs This closes #2571. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5915613d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5915613d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5915613d Branch: refs/heads/flip-6 Commit: 5915613df00f46fa493fd7d3ff63c836d9fd6146 Parents: 30517da Author: Maximilian Michels Authored: Thu Sep 29 15:08:32 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../clusterframework/types/ResourceSlot.java| 12 +- .../resourcemanager/ResourceManager.java| 115 +-- .../resourcemanager/ResourceManagerGateway.java | 31 +- .../ResourceManagerServices.java| 2 +- .../resourcemanager/SlotRequestRegistered.java | 33 -- .../resourcemanager/SlotRequestRejected.java| 34 -- .../resourcemanager/SlotRequestReply.java | 41 --- .../StandaloneResourceManager.java | 8 +- .../jobmanager/RMSlotRequestRegistered.java | 33 ++ .../jobmanager/RMSlotRequestRejected.java | 34 ++ .../messages/jobmanager/RMSlotRequestReply.java | 41 +++ .../taskexecutor/SlotAvailableReply.java| 47 +++ .../taskexecutor/TMSlotRequestRegistered.java | 35 ++ .../taskexecutor/TMSlotRequestRejected.java | 35 ++ .../taskexecutor/TMSlotRequestReply.java| 58 .../registration/TaskExecutorRegistration.java | 12 +- .../registration/WorkerRegistration.java| 42 +++ .../slotmanager/SimpleSlotManager.java | 53 --- .../slotmanager/SlotManager.java| 326 +++ .../slotmanager/SlotManagerFactory.java | 31 ++ .../flink/runtime/taskexecutor/SlotReport.java | 19 +- .../runtime/taskexecutor/TaskExecutor.java | 37 ++- .../taskexecutor/TaskExecutorGateway.java | 7 +- ...TaskExecutorToResourceManagerConnection.java | 2 +- .../resourcemanager/ResourceManagerHATest.java | 12 +- .../ResourceManagerJobMasterTest.java | 4 +- .../ResourceManagerTaskExecutorTest.java| 53 +-- .../resourcemanager/TestingResourceManager.java | 53 +++ .../resourcemanager/TestingSlotManager.java | 78 + .../slotmanager/SlotManagerTest.java| 239 ++ .../slotmanager/SlotProtocolTest.java | 92 -- .../runtime/taskexecutor/TaskExecutorTest.java | 96 +- 32 files changed, 1087 insertions(+), 628 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/5915613d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java index 4a91a79..0b9367d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.clusterframewo
[34/50] [abbrv] flink git commit: [hotfix] Replace TaskManager.createTaskManagerComponents by TaskManagerServices
[hotfix] Replace TaskManager.createTaskManagerComponents by TaskManagerServices Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bce292ae Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bce292ae Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bce292ae Branch: refs/heads/flip-6 Commit: bce292ae9eb15ba07598e26c96634c7eee45db9d Parents: 1f198d8 Author: Till Rohrmann Authored: Wed Sep 28 14:04:54 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../clusterframework/MesosTaskManager.scala | 3 +- .../taskexecutor/TaskManagerConfiguration.java | 25 +- .../TaskManagerServicesConfiguration.java | 2 +- .../minicluster/LocalFlinkMiniCluster.scala | 47 +- .../flink/runtime/taskmanager/TaskManager.scala | 605 ++- .../taskmanager/TaskManagerConfiguration.scala | 56 -- ...askManagerComponentsStartupShutdownTest.java | 24 +- .../testingUtils/TestingTaskManager.scala | 3 +- .../runtime/testingUtils/TestingUtils.scala | 1 - .../flink/yarn/TestingYarnTaskManager.scala | 3 +- .../org/apache/flink/yarn/YarnTaskManager.scala | 3 +- 11 files changed, 126 insertions(+), 646 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala -- diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index 3972a57..e8d6a58 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -24,7 +24,8 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.MetricRegistry -import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation} +import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration +import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} /** An extension of the TaskManager that listens for additional Mesos-related * messages. http://git-wip-us.apache.org/repos/asf/flink/blob/bce292ae/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index 32eb8c1..f58af77 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -41,6 +41,7 @@ public class TaskManagerConfiguration { private final String[] tmpDirPaths; private final Time timeout; + // null indicates an infinite duration private final Time maxRegistrationDuration; private final Time initialRegistrationPause; private final Time maxRegistrationPause; @@ -48,6 +49,9 @@ public class TaskManagerConfiguration { private final long cleanupInterval; + // TODO: remove necessity for complete configuration object + private final Configuration configuration; + public TaskManagerConfiguration( int numberSlots, String[] tmpDirPaths, @@ -56,16 +60,18 @@ public class TaskManagerConfiguration { Time initialRegistrationPause, Time maxRegistrationPause, Time refusedRegistrationPause, - long cleanupInterval) { + long cleanupInterval, + Configuration configuration) { this.numberSlots = numberSlots; this.tmpDirPaths = Preconditions.checkNotNull(tmpDirPaths); this.timeout = Preconditions.checkNotNull(timeout); - this.maxRegistrationDuration = Preconditions.checkNotNull(maxRegistrationDuration); + this.maxRegistrationDuration = maxRegistrationDuration; this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause); this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause); this.refusedRegistrationPau
[41/50] [abbrv] flink git commit: [FLINK-4735] [cluster management] Implements some job execution related RPC calls on the JobManager
[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/041dfd78 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/041dfd78 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/041dfd78 Branch: refs/heads/flip-6 Commit: 041dfd78a70a45a2b697029c8c1e914050ffee91 Parents: cef3191 Author: Kurt Young Authored: Tue Oct 4 23:00:22 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:43 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/041dfd78/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.Se
[04/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/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 new file mode 100644 index 000..4871b96 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -0,0 +1,827 @@ +/* + * 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.taskexecutor; + +import akka.actor.ActorSystem; +import akka.dispatch.ExecutionContexts$; +import akka.util.Timeout; +import com.typesafe.config.Config; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.core.memory.HeapMemorySegment; +import org.apache.flink.core.memory.HybridMemorySegment; +import org.apache.flink.core.memory.MemorySegmentFactory; +import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.instance.InstanceConnectionInfo; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.netty.NettyConfig; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +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.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.taskmanager.MemoryLogger; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.util.MathUtils; +import org.apache.flink.util.NetUtils; + +import scala.Tuple2; +import scala.Option; +import scala.Some; +import scala.concurrent.ExecutionContext; +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.UUID; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * TaskExecutor implementation. The task executor is responsible for the execution of multiple + * {@link org.apache.flink.runtime.taskmanager.Task}. + */ +public class TaskExecutor extends RpcEndpoint { + + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class); + + /** The unique resource ID of this TaskExecutor */ + private final ResourceID resourceID; + + /** The access to the leader election and metadata storage services */ + private final HighAvailabilityServices haServices; + + /** The task manager configuration */ + private final TaskExecutorConfiguration taskExecutorConfig; + + /** The I/O manager component in the task manager */ + private final IOManager ioManager; + + /** The memory manager component in the task manager */ + private final MemoryManager memoryManager; + + /** The network component in the task manager */ + private
[02/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java new file mode 100644 index 000..52d9d06 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java @@ -0,0 +1,538 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.api.common.JobID; +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.clusterframework.types.ResourceSlot; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.taskexecutor.SlotStatus; +import org.junit.Before; +import org.junit.Test; + +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + +public class SlotManagerTest { + + private static final double DEFAULT_TESTING_CPU_CORES = 1.0; + + private static final long DEFAULT_TESTING_MEMORY = 512; + + private static final ResourceProfile DEFAULT_TESTING_PROFILE = + new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY); + + private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE = + new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2); + + private ResourceManagerGateway resourceManagerGateway; + + @Before + public void setUp() { + resourceManagerGateway = mock(ResourceManagerGateway.class); + } + + /** +* Tests that there are no free slots when we request, need to allocate from cluster manager master +*/ + @Test + public void testRequestSlotWithoutFreeSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + + assertEquals(0, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(1, slotManager.getPendingRequestCount()); + assertEquals(1, slotManager.getAllocatedContainers().size()); + assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0)); + } + + /** +* Tests that there are some free slots when we request, and the request is fulfilled immediately +*/ + @Test + public void testRequestSlotWithFreeSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1); + assertEquals(1, slotManager.getFreeSlotCount()); + + slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE)); + assertEquals(1, slotManager.getAllocatedSlotCount()); + assertEquals(0, slotManager.getFreeSlotCount()); + assertEquals(0, slotManager.getPendingRequestCount()); + assertEquals(0, slotManager.getAllocatedContainers().size()); + } + + /** +* Tests that there are some free slots when we request, but none of them are suitable +*/ + @Test + public void testRequestSlotWithoutSuitableSlot() { + TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway); + + directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2); +
[13/50] [abbrv] flink git commit: [FLINK-4537] [cluster management] ResourceManager registration with JobManager
[FLINK-4537] [cluster management] ResourceManager registration with JobManager Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/efc7de5b Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/efc7de5b Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/efc7de5b Branch: refs/heads/flip-6 Commit: efc7de5bd3bff0512c20485f94d563c9e9cea5ec Parents: f4dc474 Author: beyond1920 Authored: Thu Sep 1 15:27:20 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:40 2016 +0200 -- .../HighAvailabilityServices.java | 9 ++ .../runtime/highavailability/NonHaServices.java | 19 +++ .../jobmaster/JobMasterRegistrationSuccess.java | 49 ++ .../resourcemanager/JobMasterRegistration.java | 39 - .../resourcemanager/ResourceManager.java| 125 +-- .../resourcemanager/ResourceManagerGateway.java | 34 ++-- .../exceptions/LeaderSessionIDException.java| 60 +++ .../runtime/taskexecutor/TaskExecutor.java | 5 + .../TestingHighAvailabilityServices.java| 17 ++ .../resourcemanager/ResourceManagerTest.java| 160 +++ 10 files changed, 483 insertions(+), 34 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 298147c..7634176 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 @@ -40,6 +40,15 @@ public interface HighAvailabilityServices { LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception; /** +* Gets the leader retriever for the job JobMaster which is responsible for the given job +* +* @param jobID The identifier of the job. +* @return +* @throws Exception +*/ + LeaderRetrievalService getJobMasterLeaderRetriever(JobID jobID) throws Exception; + + /** * Gets the leader election service for the cluster's resource manager. * @return * @throws Exception http://git-wip-us.apache.org/repos/asf/flink/blob/efc7de5b/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 292a404..33dc2d7 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,6 +19,7 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +import org.apache.flink.hadoop.shaded.org.jboss.netty.util.internal.ConcurrentHashMap; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -42,6 +43,8 @@ public class NonHaServices implements HighAvailabilityServices { /** The fix address of the ResourceManager */ private final String resourceManagerAddress; + private final ConcurrentHashMap jobMastersAddress; + /** * Creates a new services class for the fix pre-defined leaders. * @@ -49,6 +52,17 @@ public class NonHaServices implements HighAvailabilityServices { */ public NonHaServices(String resourceManagerAddress) { this.resourceManagerAddress = checkNotNull(resourceManagerAddress); + this.jobMastersAddress = new ConcurrentHashMap<>(16); + } + + /** +* Binds address of a specified job master +* +* @param jobIDJobID for the specified job master +* @param jobMasterAddress address for the specified job master +*/ + public void bindJobMasterLeaderAddress(JobID jobID, String jobMasterAddress) { + jobMastersAddress.put(jobID, jobMasterAddress); } // @@ -61,6 +75,11 @@ public class NonHaServices implements HighAvailabilityServices {
[30/50] [abbrv] flink git commit: [FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable
[FLINK-4530] [rpc] Generalize TaskExecutorToResourceManagerConnection to be reusable This closes #2520 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b8c6b998 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b8c6b998 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b8c6b998 Branch: refs/heads/flip-6 Commit: b8c6b9986522aea96c79c774eeea5dc57a7bfc64 Parents: 07512e0 Author: zhuhaifengleon Authored: Mon Sep 26 17:43:44 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../JobMasterToResourceManagerConnection.java | 117 +++ .../registration/RegisteredRpcConnection.java | 192 +++ .../runtime/taskexecutor/TaskExecutor.java | 4 +- ...TaskExecutorToResourceManagerConnection.java | 127 +++- .../RegisteredRpcConnectionTest.java| 183 ++ .../registration/RetryingRegistrationTest.java | 6 +- 6 files changed, 519 insertions(+), 110 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/b8c6b998/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java new file mode 100644 index 000..71fce8c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterToResourceManagerConnection.java @@ -0,0 +1,117 @@ +/* + * 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; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +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.RpcService; +import org.apache.flink.runtime.concurrent.Future; + +import org.slf4j.Logger; + +import java.util.UUID; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * The connection between a JobMaster and the ResourceManager. + */ +public class JobMasterToResourceManagerConnection + extends RegisteredRpcConnection { + + /** the JobMaster whose connection to the ResourceManager this represents */ + private final JobMaster jobMaster; + + private final JobID jobID; + + private final UUID jobMasterLeaderId; + + public JobMasterToResourceManagerConnection( + Logger log, + JobID jobID, + JobMaster jobMaster, + UUID jobMasterLeaderId, + String resourceManagerAddress, + UUID resourceManagerLeaderId, + Executor executor) { + + super(log, resourceManagerAddress, resourceManagerLeaderId, executor); + this.jobMaster = checkNotNull(jobMaster); + this.jobID = checkNotNull(jobID); + this.jobMasterLeaderId = checkNotNull(jobMasterLeaderId); + } + + @Override + protected RetryingRegistration generateRegistration() { + return new JobMasterToResourceManagerConnection.ResourceManagerRegistration( + log, jobMaster.getRpcService(), + getTargetAddress(), getTargetLeaderId(), + jobMaster.getAddress(),jobID, jobMasterLeaderId); + } + + @Override + protected void onRegistrationSuccess(JobMasterRegistrationSuccess success) { + } + + @Override + protected void onRegistrationFailure(Throwable failure) { + } + +
[32/50] [abbrv] flink git commit: [FLINK-4606] integrate features of old ResourceManager
[FLINK-4606] integrate features of old ResourceManager This closes #2540 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/1f198d8c Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/1f198d8c Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/1f198d8c Branch: refs/heads/flip-6 Commit: 1f198d8ca56ec6719d112cdc7180aeef6d18477a Parents: 6e58ebf Author: Maximilian Michels Authored: Tue Sep 27 10:38:02 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../InfoMessageListenerRpcGateway.java | 1 - .../resourcemanager/ResourceManager.java| 146 --- .../resourcemanager/ResourceManagerGateway.java | 6 +- .../ResourceManagerServices.java| 44 ++ .../StandaloneResourceManager.java | 19 ++- .../TaskExecutorRegistration.java | 51 --- .../registration/TaskExecutorRegistration.java | 51 +++ .../slotmanager/SimpleSlotManager.java | 6 - .../slotmanager/SlotManager.java| 63 ++-- .../slotmanager/SlotManagerTest.java| 25 +++- .../slotmanager/SlotProtocolTest.java | 42 +++--- 11 files changed, 295 insertions(+), 159 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java index c1eeefa..d1373ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.runtime.clusterframework.messages.InfoMessage; -import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.rpc.RpcGateway; /** http://git-wip-us.apache.org/repos/asf/flink/blob/1f198d8c/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 83dc4db..190a4de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -20,14 +20,18 @@ package org.apache.flink.runtime.resourcemanager; 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.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.messages.InfoMessage; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable; +import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.concurrent.AcceptFunction; 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.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -48,11 +52,10 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import scala.concurrent.duration.FiniteDuration; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.UUID; import java.util.concurrent.Callable; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -64,36 +67,43 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * It offers the following methods as part of its rpc interface to interact with the him remotely: * * {@link #registerJobMaster(UUID, UUID, String, JobID)} registers a {@link JobMaster} at the resource manager - * {@link #requestSlot(SlotRequest)} requests a slot
[31/50] [abbrv] flink git commit: [FLINK-4703] RpcCompletenessTest: Add support for type arguments and subclasses
[FLINK-4703] RpcCompletenessTest: Add support for type arguments and subclasses This closes #2561 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6e58ebf2 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6e58ebf2 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6e58ebf2 Branch: refs/heads/flip-6 Commit: 6e58ebf22cb11631438ea51118615053e11cbcdb Parents: 415af17 Author: Maximilian Michels Authored: Wed Sep 28 12:39:30 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../apache/flink/runtime/rpc/RpcEndpoint.java | 23 +- .../flink/runtime/rpc/RpcCompletenessTest.java | 80 ++-- 2 files changed, 94 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6e58ebf2/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 4e5e49a..79961f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -85,9 +85,9 @@ public abstract class RpcEndpoint { // IMPORTANT: Don't change order of selfGatewayType and self because rpcService.startServer // requires that selfGatewayType has been initialized - this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass()); + this.selfGatewayType = determineSelfGatewayType(); this.self = rpcService.startServer(this); - + this.mainThreadExecutor = new MainThreadExecutor((MainThreadExecutable) self); } @@ -255,4 +255,23 @@ public abstract class RpcEndpoint { gateway.runAsync(runnable); } } + + /** +* Determines the self gateway type specified in one of the subclasses which extend this class. +* May traverse multiple class hierarchies until a Gateway type is found as a first type argument. +* @return Class The determined self gateway type +*/ + private Class determineSelfGatewayType() { + + // determine self gateway type + Class c = getClass(); + Class determinedSelfGatewayType; + do { + determinedSelfGatewayType = ReflectionUtil.getTemplateType1(c); + // check if super class contains self gateway type in next loop + c = c.getSuperclass(); + } while (!RpcGateway.class.isAssignableFrom(determinedSelfGatewayType)); + + return determinedSelfGatewayType; + } } http://git-wip-us.apache.org/repos/asf/flink/blob/6e58ebf2/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java -- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java index 53355e8..e7143ae 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java @@ -26,9 +26,14 @@ import org.apache.flink.util.ReflectionUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; import org.reflections.Reflections; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.lang.annotation.Annotation; import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -41,8 +46,33 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +/** + * Test which ensures that all classes of subtype {@link RpcEndpoint} implement + * the methods specified in the generic gateway type argument. + * + * {@code + * RpcEndpoint + * } + * + * Note, that the class hierarchy can also be nested. In this case the type argument + * always has to be the first argument, e.g. {@code + * + * // RpcClass needs to implement RpcGatewayClass' methods + * RpcClass extends RpcEndpoint + * + * // RpcClass2 or its subclass needs to implement RpcGatewayClass' methods + * RpcClass extends RpcEndpoint + * RpcClass2 extends RpcClass + * + * // needless to say, this can even be nested fur
[01/50] [abbrv] flink git commit: [FLINK-4347][cluster management] Implement SlotManager core [Forced Update!]
Repository: flink Updated Branches: refs/heads/flip-6 48c936eed -> 7aca811df (forced update) [FLINK-4347][cluster management] Implement SlotManager core This closes #2388 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6e22c64d Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6e22c64d Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6e22c64d Branch: refs/heads/flip-6 Commit: 6e22c64dc6f5c2c252df46a5a32adf20da7ef162 Parents: ce6114d Author: Kurt Young Authored: Thu Aug 18 15:48:30 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:39 2016 +0200 -- .../runtime/clusterframework/SlotManager.java | 525 ++ .../clusterframework/types/ResourceID.java | 4 +- .../clusterframework/types/ResourceSlot.java| 66 +++ .../runtime/clusterframework/types/SlotID.java | 14 +- .../rpc/resourcemanager/SlotRequest.java| 51 +- .../runtime/rpc/taskexecutor/SlotReport.java| 56 ++ .../runtime/rpc/taskexecutor/SlotStatus.java| 129 + .../clusterframework/SlotManagerTest.java | 540 +++ 8 files changed, 1377 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/6e22c64d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java new file mode 100644 index 000..cc140a1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java @@ -0,0 +1,525 @@ +/* + * 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.clusterframework; + +import org.apache.flink.annotation.VisibleForTesting; +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.clusterframework.types.ResourceSlot; +import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest; +import org.apache.flink.runtime.rpc.taskexecutor.SlotReport; +import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request + * slots from registered TaskManagers and issues container allocation requests in case of there are not + * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat. + * + * The main operation principle of SlotManager is: + * + * 1. All slot allocation status should be synced with TaskManager, which is the ground truth. + * 2. All slots that have registered must be tracked, either by free pool or allocated pool. + * 3. All slot requests will be handled by best efforts, there is no guarantee that one request will be + * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should + * be handled outside SlotManager. SlotManager will make each decision based on the information it currently + * holds. + * + * IMPORTANT: This class is Not Thread-safe. + */ +public abstract class SlotManager { + + private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class); + + /** Gateway to communicate with ResourceManager */ + private final ResourceManagerGateway resourceManagerGateway; + + /** All registered slots, including free and allocated slots */ +
[24/50] [abbrv] flink git commit: [FLINK-4535] [cluster management] resourceManager process the registration from TaskExecutor
[FLINK-4535] [cluster management] resourceManager process the registration from TaskExecutor Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c9764c8f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c9764c8f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c9764c8f Branch: refs/heads/flip-6 Commit: c9764c8f62e08a1fbf929ee2b718b52b8ae4fff2 Parents: 34a6854 Author: beyond1920 Authored: Thu Sep 1 11:14:00 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../resourcemanager/RegistrationResponse.java | 36 --- .../resourcemanager/ResourceManager.java| 288 --- .../resourcemanager/ResourceManagerGateway.java | 45 +-- .../TaskExecutorRegistration.java | 51 .../exceptions/LeaderSessionIDException.java| 1 + .../resourcemanager/ResourceManagerTest.java| 119 6 files changed, 241 insertions(+), 299 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java deleted file mode 100644 index 796e634..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java +++ /dev/null @@ -1,36 +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.resourcemanager; - -import java.io.Serializable; - -public class RegistrationResponse implements Serializable { - private static final long serialVersionUID = -2379003255993119993L; - - private final boolean isSuccess; - - public RegistrationResponse(boolean isSuccess) { - this.isSuccess = isSuccess; - } - - public boolean isSuccess() { - return isSuccess; - } - -} http://git-wip-us.apache.org/repos/asf/flink/blob/c9764c8f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index aae4874..15692b6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -18,41 +18,29 @@ package org.apache.flink.runtime.resourcemanager; +import akka.dispatch.Futures; +import akka.dispatch.Mapper; + import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.ApplyFunction; -import org.apache.flink.runtime.concurrent.Future; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcMethod; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDExce
[20/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner
http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java deleted file mode 100644 index 4f756fb..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerRunner.java +++ /dev/null @@ -1,749 +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.taskmanager; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.memory.HeapMemorySegment; -import org.apache.flink.core.memory.HybridMemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.ConnectionManager; -import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.query.KvStateRegistry; -import org.apache.flink.runtime.query.netty.DisabledKvStateRequestStats; -import org.apache.flink.runtime.query.netty.KvStateServer; -import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.taskexecutor.TaskExecutor; -import org.apache.flink.runtime.taskexecutor.TaskExecutorConfiguration; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.util.MathUtils; -import org.apache.flink.util.NetUtils; - -import akka.actor.ActorSystem; -import akka.util.Timeout; -import com.typesafe.config.Config; -import org.apache.flink.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.Option; -import scala.Some; -import scala.Tuple2; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.FiniteDuration; - -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * This class is the executable entry point for the task manager in yarn or standalone mode. - * It constructs the related components (network, I/O manager, memory manager, RPC service, HA service) - * and starts them. - */ -public class TaskManagerRunner { - - private static final Logger LOG = LoggerFactory.getLogger(TaskManagerRunner.class); - - /** -* Constructs related components of the TaskManager and starts them. -* -* @param configuration The configuration for the TaskManager. -* @param resourceIDThe id of the resource which the task manager will run on. -* @param rpcServiceOptionally, The rpc service which is used to start and connect to the TaskManager RpcEndpoint . -
[21/50] [abbrv] flink git commit: [FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner
[FLINK-4505] [cluster mngt] Separate TaskManager service configuration from TaskManagerConfiguration; Implement TaskManagerRunner Refactors the startup logic so that is easier to reuse. This closes #2461. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bb781aef Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bb781aef Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bb781aef Branch: refs/heads/flip-6 Commit: bb781aef10b6c7099c83678a95c1d0db79cdbe3d Parents: c34f13c Author: Till Rohrmann Authored: Wed Sep 21 12:33:15 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../HighAvailabilityServicesUtils.java | 41 + .../flink/runtime/rpc/RpcServiceUtils.java | 73 ++ .../flink/runtime/rpc/akka/AkkaRpcService.java | 2 + .../runtime/taskexecutor/TaskExecutor.java | 51 +- .../taskexecutor/TaskExecutorConfiguration.java | 142 .../taskexecutor/TaskManagerConfiguration.java | 205 + .../runtime/taskexecutor/TaskManagerRunner.java | 172 + .../taskexecutor/TaskManagerServices.java | 320 .../TaskManagerServicesConfiguration.java | 325 .../runtime/taskmanager/TaskManagerRunner.java | 749 --- .../runtime/util/LeaderRetrievalUtils.java | 7 + .../apache/flink/runtime/akka/AkkaUtils.scala | 4 + .../NetworkEnvironmentConfiguration.scala | 2 +- .../flink/runtime/taskmanager/TaskManager.scala | 6 +- .../io/network/NetworkEnvironmentTest.java | 4 +- .../runtime/rpc/TestingSerialRpcService.java| 1 - .../runtime/taskexecutor/TaskExecutorTest.java | 29 +- ...askManagerComponentsStartupShutdownTest.java | 3 +- .../TaskManagerConfigurationTest.java | 1 - 19 files changed, 1195 insertions(+), 942 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java new file mode 100644 index 000..f3da847 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java @@ -0,0 +1,41 @@ +/* + * 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.configuration.Configuration; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; + +public class HighAvailabilityServicesUtils { + + public static HighAvailabilityServices createHighAvailabilityServices(Configuration configuration) throws Exception { + HighAvailabilityMode highAvailabilityMode = LeaderRetrievalUtils.getRecoveryMode(configuration); + + switch(highAvailabilityMode) { + case NONE: + final String resourceManagerAddress = null; + return new NonHaServices(resourceManagerAddress); + case ZOOKEEPER: + throw new UnsupportedOperationException("ZooKeeper high availability services " + + "have not been implemented yet."); + default: + throw new Exception("Recovery mode " + highAvailabilityMode + " is not supported."); + } + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/bb781aef/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcServiceUtils.java b/flink-runtime/
[05/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package
http://git-wip-us.apache.org/repos/asf/flink/blob/9c247d1f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java deleted file mode 100644 index 36d6310..000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java +++ /dev/null @@ -1,827 +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.rpc.taskexecutor; - -import akka.actor.ActorSystem; -import akka.dispatch.ExecutionContexts$; -import akka.util.Timeout; -import com.typesafe.config.Config; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.core.memory.HeapMemorySegment; -import org.apache.flink.core.memory.HybridMemorySegment; -import org.apache.flink.core.memory.MemorySegmentFactory; -import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceConnectionInfo; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.netty.NettyConfig; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; -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.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.taskmanager.MemoryLogger; -import org.apache.flink.runtime.util.EnvironmentInformation; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; -import org.apache.flink.util.MathUtils; -import org.apache.flink.util.NetUtils; - -import scala.Tuple2; -import scala.Option; -import scala.Some; -import scala.concurrent.ExecutionContext; -import scala.concurrent.duration.Duration; -import scala.concurrent.duration.FiniteDuration; - -import java.io.File; -import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.util.UUID; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.TimeUnit; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * TaskExecutor implementation. The task executor is responsible for the execution of multiple - * {@link org.apache.flink.runtime.taskmanager.Task}. - */ -public class TaskExecutor extends RpcEndpoint { - - private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class); - - /** The unique resource ID of this TaskExecutor */ - private final ResourceID resourceID; - - /** The access to the leader election and metadata storage services */ - private final HighAvailabilityServices haServices; - - /** The task manager configuration */ - private final TaskExecutorConfiguration taskExecutorConfig; - - /** The I/O manager component in the task manager */ - private final IOManager ioManager; - - /** The memory manager component in the task manager */ - private final MemoryManager memoryManager; - - /** The network component in the task ma
[17/50] [abbrv] flink git commit: [FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol
[FLINK-4538][FLINK-4348] ResourceManager slot allocation protcol - associates JobMasters with JobID instead of InstanceID - adds TaskExecutorGateway to slot - adds SlotManager as RM constructor parameter - adds LeaderRetrievalListener to SlotManager to keep track of the leader id - tests the interaction JM->RM requestSlot - tests the interaction RM->TM requestSlot This closes #2463 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/74570d45 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/74570d45 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/74570d45 Branch: refs/heads/flip-6 Commit: 74570d454e61a890db4e830b4b71caa178b31bed Parents: 9c07278 Author: Maximilian Michels Authored: Thu Sep 1 16:53:31 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:40 2016 +0200 -- .../clusterframework/types/ResourceProfile.java | 2 +- .../clusterframework/types/ResourceSlot.java| 14 +- .../resourcemanager/JobMasterRegistration.java | 10 +- .../resourcemanager/RegistrationResponse.java | 9 +- .../resourcemanager/ResourceManager.java| 167 +++--- .../resourcemanager/ResourceManagerGateway.java | 2 +- .../runtime/resourcemanager/SlotAssignment.java | 25 - .../runtime/resourcemanager/SlotManager.java| 523 - .../resourcemanager/SlotRequestRegistered.java | 33 ++ .../resourcemanager/SlotRequestRejected.java| 34 ++ .../resourcemanager/SlotRequestReply.java | 41 ++ .../slotmanager/SimpleSlotManager.java | 59 ++ .../slotmanager/SlotManager.java| 579 +++ .../flink/runtime/taskexecutor/SlotStatus.java | 5 +- .../taskexecutor/TaskExecutorGateway.java | 17 + .../resourcemanager/ResourceManagerHATest.java | 4 +- .../resourcemanager/SlotManagerTest.java| 538 - .../slotmanager/SlotManagerTest.java| 554 ++ .../slotmanager/SlotProtocolTest.java | 225 +++ .../flink/runtime/rpc/TestingRpcService.java| 6 +- .../runtime/rpc/TestingSerialRpcService.java| 4 + 21 files changed, 1677 insertions(+), 1174 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/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 4e78721..7a25de1 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 @@ -87,7 +87,7 @@ public class ResourceProfile implements Serializable { * @return true if the requirement is matched, otherwise false */ public boolean isMatching(ResourceProfile required) { - return Double.compare(cpuCores, required.getCpuCores()) >= 0 && memoryInMB >= required.getMemoryInMB(); + return cpuCores >= required.getCpuCores() && memoryInMB >= required.getMemoryInMB(); } // http://git-wip-us.apache.org/repos/asf/flink/blob/74570d45/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java index 8a6db5f..5fb8aee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.clusterframework.types; +import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; + import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -26,7 +28,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique * identification and resource profile which we can compare to the resource request. */ -public class ResourceSlot implements ResourceIDRetrievable, Serializable { +public class ResourceSlot implements ResourceIDRetrievable { private static final long serialVersionUID = -5853720153136840674L; @@ -36,9 +3
[42/50] [abbrv] flink git commit: [FLINK-4694] [rpc] Add termination futures to RpcEndpoint and RpcService
[FLINK-4694] [rpc] Add termination futures to RpcEndpoint and RpcService The termination futures can be used to wait for the termination of the respective component. This closes #2558. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9dfaf457 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9dfaf457 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9dfaf457 Branch: refs/heads/flip-6 Commit: 9dfaf457fcc282fb01a1ee11950416e6a0b51171 Parents: 5915613 Author: Till Rohrmann Authored: Tue Sep 27 18:17:42 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:43 2016 +0200 -- .../concurrent/impl/FlinkCompletableFuture.java | 11 +++--- .../apache/flink/runtime/rpc/RpcEndpoint.java | 9 + .../apache/flink/runtime/rpc/RpcService.java| 7 .../apache/flink/runtime/rpc/SelfGateway.java | 34 ++ .../runtime/rpc/akka/AkkaInvocationHandler.java | 22 ++-- .../flink/runtime/rpc/akka/AkkaRpcActor.java| 17 - .../flink/runtime/rpc/akka/AkkaRpcService.java | 32 +++-- .../runtime/rpc/TestingSerialRpcService.java| 10 +- .../runtime/rpc/akka/AkkaRpcActorTest.java | 36 .../runtime/rpc/akka/AkkaRpcServiceTest.java| 29 10 files changed, 193 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java index e648a71..14686d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/impl/FlinkCompletableFuture.java @@ -20,7 +20,6 @@ package org.apache.flink.runtime.concurrent.impl; import akka.dispatch.Futures; import org.apache.flink.runtime.concurrent.CompletableFuture; -import org.apache.flink.util.Preconditions; import scala.concurrent.Promise; import scala.concurrent.Promise$; @@ -52,8 +51,6 @@ public class FlinkCompletableFuture extends FlinkFuture implements Complet @Override public boolean complete(T value) { - Preconditions.checkNotNull(value); - try { promise.success(value); @@ -65,10 +62,12 @@ public class FlinkCompletableFuture extends FlinkFuture implements Complet @Override public boolean completeExceptionally(Throwable t) { - Preconditions.checkNotNull(t); - try { - promise.failure(t); + if (t == null) { + promise.failure(new NullPointerException("Throwable was null.")); + } else { + promise.failure(t); + } return true; } catch (IllegalStateException e) { http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 79961f7..f93a2e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -173,6 +173,15 @@ public abstract class RpcEndpoint { return rpcService; } + /** +* Return a future which is completed when the rpc endpoint has been terminated. +* +* @return Future which is completed when the rpc endpoint has been terminated. +*/ + public Future getTerminationFuture() { + return ((SelfGateway)self).getTerminationFuture(); + } + // // Asynchronous executions // http://git-wip-us.apache.org/repos/asf/flink/blob/9dfaf457/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 96844ed..2
[49/50] [abbrv] flink git commit: [FLINK-4375] [distributed coordination] Implement new JobManager creation, initialization, and basic RPC methods
[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/0615b62f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/0615b62f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/0615b62f Branch: refs/heads/flip-6 Commit: 0615b62fb2239f3353e450768f820e5032c0cb7e Parents: 041dfd7 Author: Kurt Young Authored: Wed Oct 12 23:25:16 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:32:27 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 | 2 +- .../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 | 269 -- .../runtime/jobmaster/JobManagerServices.java | 86 +- .../flink/runtime/jobmaster/JobMaster.java | 900 +++ .../runtime/jobmaster/JobMasterGateway.java | 115 +-- .../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 ++- .../rpc/RpcCheckpointResponder.java | 2 +- .../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 + .../runtime/operators/DataSinkTaskTest.java | 2 +- 33 files changed, 1252 insertions(+), 1006 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/0615b62f/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/0615b62f/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
[36/50] [abbrv] flink git commit: [FLINK-4478] [flip-6] Add HeartbeatManager
[FLINK-4478] [flip-6] Add HeartbeatManager Add a heartbeat manager abstraction which can monitor heartbeat targets. Whenever no heartbeat signal has been received for a heartbeat timeout interval, the heartbeat manager will issue a heartbeat timeout notification. Add resourceID to HeartbeatListener.reportPayload Replace scala future by Flink's futures Add unmonitoring test This closes #2435. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3e4eb4f9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3e4eb4f9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3e4eb4f9 Branch: refs/heads/flip-6 Commit: 3e4eb4f92012265b6fff27f0544fcd6d1629431f Parents: 214113e Author: Till Rohrmann Authored: Thu Aug 25 14:05:07 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../runtime/heartbeat/HeartbeatListener.java| 62 .../runtime/heartbeat/HeartbeatManager.java | 67 .../runtime/heartbeat/HeartbeatManagerImpl.java | 328 +++ .../heartbeat/HeartbeatManagerSenderImpl.java | 81 + .../runtime/heartbeat/HeartbeatTarget.java | 50 +++ .../runtime/heartbeat/HeartbeatManagerTest.java | 315 ++ .../slotmanager/SlotProtocolTest.java | 4 - 7 files changed, 903 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java new file mode 100644 index 000..8c08251 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.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.heartbeat; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Future; + +/** + * Interface for the interaction with the {@link HeartbeatManager}. The heartbeat listener is used + * for the following things: + * + * + * Notifications about heartbeat timeouts + * Payload reports of incoming heartbeats + * Retrieval of payloads for outgoing heartbeats + * + * @param Type of the incoming payload + * @param Type of the outgoing payload + */ +public interface HeartbeatListener { + + /** +* Callback which is called if a heartbeat for the machine identified by the given resource +* ID times out. +* +* @param resourceID Resource ID of the machine whose heartbeat has timed out +*/ + void notifyHeartbeatTimeout(ResourceID resourceID); + + /** +* Callback which is called whenever a heartbeat with an associated payload is received. The +* carried payload is given to this method. +* +* @param resourceID Resource ID identifying the sender of the payload +* @param payload Payload of the received heartbeat +*/ + void reportPayload(ResourceID resourceID, I payload); + + /** +* Retrieves the payload value for the next heartbeat message. Since the operation can happen +* asynchronously, the result is returned wrapped in a future. +* +* @return Future containing the next payload for heartbeats +*/ + Future retrievePayload(); +} http://git-wip-us.apache.org/repos/asf/flink/blob/3e4eb4f9/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManager.java new file mode 100644 index 000..12918ed --- /dev/nul
[27/50] [abbrv] flink git commit: [FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph
[FLINK-4408] [JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph This closes #2480 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/360eaf8a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/360eaf8a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/360eaf8a Branch: refs/heads/flip-6 Commit: 360eaf8a764ef981739f56598bbfd742754456f2 Parents: 0e495b7 Author: Kurt Young Authored: Thu Sep 8 12:00:13 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:41 2016 +0200 -- .../flink/api/common/JobExecutionResult.java| 2 +- .../flink/api/common/JobSubmissionResult.java | 2 +- .../HighAvailabilityServices.java | 12 + .../runtime/highavailability/NonHaServices.java | 16 +- .../runtime/jobmanager/OnCompletionActions.java | 31 ++ .../runtime/jobmanager/scheduler/Scheduler.java | 9 + .../runtime/jobmaster/JobManagerRunner.java | 288 +++ .../runtime/jobmaster/JobManagerServices.java | 73 +++ .../flink/runtime/jobmaster/JobMaster.java | 485 ++- .../runtime/jobmaster/JobMasterGateway.java | 13 + .../jobmaster/MiniClusterJobDispatcher.java | 385 +++ .../flink/runtime/rpc/FatalErrorHandler.java| 24 + .../runtime/taskexecutor/TaskExecutor.java | 12 + .../TestingHighAvailabilityServices.java| 39 +- .../jobmaster/JobManagerRunnerMockTest.java | 254 ++ .../flink/runtime/rpc/RpcConnectionTest.java| 17 +- 16 files changed, 1533 insertions(+), 129 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index cb4ecc5..7286cc5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit; @Public public class JobExecutionResult extends JobSubmissionResult { - private long netRuntime; + private final long netRuntime; private final Map accumulatorResults; http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java -- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java index c5dc869..b0e7e24 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobSubmissionResult.java @@ -26,7 +26,7 @@ import org.apache.flink.annotation.Public; @Public public class JobSubmissionResult { - private JobID jobID; + private final JobID jobID; public JobSubmissionResult(JobID jobID) { this.jobID = jobID; http://git-wip-us.apache.org/repos/asf/flink/blob/360eaf8a/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 7634176..d67e927 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,6 +19,8 @@ package org.apache.flink.runtime.highavailability; import org.apache.flink.api.common.JobID; +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; @@ -61,4 +63,14 @@ public interface HighAvailabilityServices { * @param jobID The identifier of the job running the election. */ LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception; + + /** +* Gets the checkpoint recovery factory for the job manager +*/ + CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception; + + /** +* Gets
[40/50] [abbrv] flink git commit: [FLINK-4606] Integrate the new ResourceManager with the existed FlinkResourceManager
[FLINK-4606] Integrate the new ResourceManager with the existed FlinkResourceManager Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/415af17f Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/415af17f Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/415af17f Branch: refs/heads/flip-6 Commit: 415af17fdf45fe157a6ee5f7187ee63e8845f168 Parents: bb781ae Author: beyond1920 Authored: Fri Sep 9 09:11:24 2016 +0800 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../InfoMessageListenerRpcGateway.java | 35 +++ .../resourcemanager/ResourceManager.java| 214 --- .../resourcemanager/ResourceManagerGateway.java | 23 ++ .../StandaloneResourceManager.java | 64 ++ .../resourcemanager/ResourceManagerHATest.java | 2 +- .../ResourceManagerJobMasterTest.java | 2 +- .../ResourceManagerTaskExecutorTest.java| 2 +- .../slotmanager/SlotProtocolTest.java | 5 +- 8 files changed, 318 insertions(+), 29 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java new file mode 100644 index 000..c1eeefa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/InfoMessageListenerRpcGateway.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.resourcemanager; + +import org.apache.flink.runtime.clusterframework.messages.InfoMessage; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.rpc.RpcGateway; + +/** + * A gateway to listen for info messages from {@link ResourceManager} + */ +public interface InfoMessageListenerRpcGateway extends RpcGateway { + + /** +* Notifies when resource manager need to notify listener about InfoMessage +* @param infoMessage +*/ + void notifyInfoMessage(InfoMessage infoMessage); +} http://git-wip-us.apache.org/repos/asf/flink/blob/415af17f/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 88b8a11..83dc4db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -20,19 +20,22 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.clusterframework.messages.InfoMessage; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.AcceptFunction; +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.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.l
[37/50] [abbrv] flink git commit: [hotfix] Fix failing JobManagerRunnerMockTest
[hotfix] Fix failing JobManagerRunnerMockTest Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/30517da9 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/30517da9 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/30517da9 Branch: refs/heads/flip-6 Commit: 30517da90f947cc3622d969dc788797bff7abe3c Parents: 3e4eb4f Author: Till Rohrmann Authored: Tue Oct 4 14:27:37 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:42 2016 +0200 -- .../jobmaster/JobManagerRunnerMockTest.java| 17 + 1 file changed, 1 insertion(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/30517da9/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 3a769bb..f8a0b6a 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 @@ -131,21 +131,6 @@ public class JobManagerRunnerMockTest { } @Test - public void testJobFinishedByOtherBeforeGrantLeadership() throws Exception { - runner.start(); - - when(submittedJobGraphStore.contains(any(JobID.class))).thenReturn(false); - runner.grantLeadership(UUID.randomUUID()); - - // runner should shutdown automatic and informed the job completion - verify(leaderElectionService).stop(); - verify(jobManager).shutDown(); - - assertTrue(jobCompletion.isJobFinished()); - assertTrue(jobCompletion.isJobFinishedByOther()); - } - - @Test public void testJobFinished() throws Exception { runner.start(); @@ -211,7 +196,7 @@ public class JobManagerRunnerMockTest { UUID leaderSessionID2 = UUID.randomUUID(); runner.grantLeadership(leaderSessionID2); - verify(jobManagerGateway, times(2)).startJob(leaderSessionID2); + verify(jobManagerGateway).startJob(leaderSessionID2); } private static class TestingOnCompletionActions implements OnCompletionActions {
[46/50] [abbrv] flink git commit: [FLINK-4516] update leadership information in ResourceManager
[FLINK-4516] update leadership information in ResourceManager The leadership information remained static for connected JobMasters. This updates it to remove stale JobMasters when they lose leadership status. This closes #2624 Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/cef31912 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/cef31912 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/cef31912 Branch: refs/heads/flip-6 Commit: cef319126ced676b5c6d08e6a963986f1dd6c5ee Parents: b380634 Author: Maximilian Michels Authored: Mon Oct 10 17:36:10 2016 +0200 Committer: Stephan Ewen Committed: Fri Oct 14 15:14:43 2016 +0200 -- .../resourcemanager/ResourceManager.java| 196 +-- .../resourcemanager/ResourceManagerGateway.java | 4 +- .../ResourceManagerServices.java| 6 + .../registration/JobMasterRegistration.java | 62 ++ .../slotmanager/SlotManager.java| 16 +- .../resourcemanager/TestingSlotManager.java | 8 + .../slotmanager/SlotManagerTest.java| 10 +- 7 files changed, 224 insertions(+), 78 deletions(-) -- http://git-wip-us.apache.org/repos/asf/flink/blob/cef31912/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java -- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index d2d00cf..8fbb34b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.concurrent.AcceptFunction; 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.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; @@ -40,6 +41,7 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestRejected; import org.apache.flink.runtime.resourcemanager.messages.jobmanager.RMSlotRequestReply; import org.apache.flink.runtime.resourcemanager.messages.taskexecutor.SlotAvailableReply; +import org.apache.flink.runtime.resourcemanager.registration.JobMasterRegistration; import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerFactory; @@ -53,17 +55,14 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; -import org.apache.flink.runtime.util.LeaderConnectionInfo; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; -import scala.concurrent.duration.FiniteDuration; import java.io.Serializable; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -85,10 +84,10 @@ public abstract class ResourceManager protected static final int EXIT_CODE_FATAL_ERROR = -13; /** All currently registered JobMasterGateways scoped by JobID. */ - private final Map jobMasterGateways; + private final Map jobMasters; - /** LeaderListeners for all registered JobMasters. */ - private final Map jobMasterLeaderRetrievalListeners; + /** LeaderListeners for all registered JobIDs. */ + private final Map leaderListeners; /** All currently registered TaskExecutors with there framework specific worker information. */ private final Map> taskExecutors; @@ -106,7 +105,7 @@ public abstract class ResourceManager private LeaderElectionService leaderElectionService; /** ResourceManager's leader session id which is updated on leader election. */ - private UUID leaderSessionID; + private volatile UUID leaderSessionID; /** All registered listeners for status updates of the Resour