[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-13 Thread asfgit
Github user asfgit closed the pull request at:

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


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155245759
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -507,6 +514,41 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   getRpcService().execute(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not recover job graph for job 
{}.", jobId, e);
+   return;
+   }
+   runAsync(() -> {
+   if (!jobManagerRunners.containsKey(jobId)) {
--- End diff --

Removed.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155245771
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -507,6 +514,41 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   getRpcService().execute(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not recover job graph for job 
{}.", jobId, e);
+   return;
+   }
+   runAsync(() -> {
+   if (!jobManagerRunners.containsKey(jobId)) {
+   
submitJob(submittedJobGraph.getJobGraph(), RpcUtils.INF_TIMEOUT);
+   }
+   });
+   });
+   }
+
+   @Override
+   public void onRemovedJobGraph(final JobID jobId) {
+   runAsync(() -> {
+   if (jobManagerRunners.containsKey(jobId)) {
--- End diff --

Removed.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155242826
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -86,122 +125,143 @@ public static void teardown() {
}
}
 
-   /**
-* Tests that we can submit a job to the Dispatcher which then spawns a
-* new JobManagerRunner.
-*/
-   @Test
-   public void testJobSubmission() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
+   @Before
+   public void setUp() throws Exception {
+   MockitoAnnotations.initMocks(this);
 
-   TestingLeaderElectionService dispatcherLeaderElectionService = 
new TestingLeaderElectionService();
-   TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
-   
haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService);
-   haServices.setSubmittedJobGraphStore(new 
StandaloneSubmittedJobGraphStore());
+   final JobVertex testVertex = new JobVertex("testVertex");
+   testVertex.setInvokableClass(NoOpInvokable.class);
+   jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+   jobGraph.setAllowQueuedScheduling(true);
 
-   HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1L);
-   JobManagerRunner jobManagerRunner = 
mock(JobManagerRunner.class);
+   fatalErrorHandler = new TestingFatalErrorHandler();
+   final HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1L);
+   submittedJobGraphStore = spy(new 
InMemorySubmittedJobGraphStore());
 
-   final JobGraph jobGraph = mock(JobGraph.class);
-   final JobID jobId = new JobID();
-   when(jobGraph.getJobID()).thenReturn(jobId);
+   dispatcherLeaderElectionService = new 
TestingLeaderElectionService();
+   jobMasterLeaderElectionService = new 
TestingLeaderElectionService();
 
-   final TestingDispatcher dispatcher = new TestingDispatcher(
+   final TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
+   
haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService);
+   haServices.setSubmittedJobGraphStore(submittedJobGraphStore);
+   haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, 
jobMasterLeaderElectionService);
+   haServices.setCheckpointRecoveryFactory(new 
StandaloneCheckpointRecoveryFactory());
+   haServices.setResourceManagerLeaderRetriever(new 
TestingLeaderRetrievalService());
+   runningJobsRegistry = haServices.getRunningJobsRegistry();
+
+   final Configuration blobServerConfig = new Configuration();
+   blobServerConfig.setString(
+   BlobServerOptions.STORAGE_DIRECTORY,
+   temporaryFolder.newFolder().getAbsolutePath());
+
+   dispatcher = new TestingDispatcher(
rpcService,
Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(),
new Configuration(),
haServices,
mock(ResourceManagerGateway.class),
-   mock(BlobServer.class),
+   new BlobServer(blobServerConfig, new VoidBlobStore()),
heartbeatServices,
-   mock(MetricRegistryImpl.class),
+   new NoOpMetricRegistry(),
fatalErrorHandler,
-   jobManagerRunner,
-   jobId);
+   TEST_JOB_ID);
 
-   try {
-   dispatcher.start();
+   dispatcher.start();
+   }
 
-   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
+   @After
+   public void tearDown() throws Exception {
+   try {
+   fatalErrorHandler.rethrowError();
+   } finally {
+   RpcUtils.terminateRpcEndpoint(dispatcher, TIMEOUT);
+   }
+   }
 
-   // wait for the leader to be elected
-   leaderFuture.get();
+   /**
+* Tests that we can submit a job to the Dispatcher which then spawns a
+* new JobManagerRunner.
+*/
+   @Test
+   public void testJobSubmission() throws Exception {
+   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
 
-   DispatcherGateway dispatcherGateway = 

[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155242900
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -86,122 +125,143 @@ public static void teardown() {
}
}
 
-   /**
-* Tests that we can submit a job to the Dispatcher which then spawns a
-* new JobManagerRunner.
-*/
-   @Test
-   public void testJobSubmission() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
+   @Before
+   public void setUp() throws Exception {
+   MockitoAnnotations.initMocks(this);
--- End diff --

Nice catch.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155242108
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemorySubmittedJobGraphStore.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testutils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * In-Memory implementation of {@link SubmittedJobGraphStore} for testing 
purposes.
+ */
+public class InMemorySubmittedJobGraphStore implements 
SubmittedJobGraphStore {
+
+   private final Map storedJobs = new 
HashMap<>();
+
+   private boolean started;
+
+   @Override
+   public synchronized void start(@Nullable SubmittedJobGraphListener 
jobGraphListener) throws Exception {
--- End diff --

Not sure if it is actually applicable. `onRemovedJobGraph` explicitly 
demands that the graph is removed by a different `SubmittedJobGraphStore` 
instance:

```
/**
 * Callback for {@link SubmittedJobGraph} instances removed by 
a different {@link
 * SubmittedJobGraphStore} instance.
 *
 * @param jobId The {@link JobID} of the removed job graph
 */
void onRemovedJobGraph(JobID jobId);
```


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155240139
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemorySubmittedJobGraphStore.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testutils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * In-Memory implementation of {@link SubmittedJobGraphStore} for testing 
purposes.
+ */
+public class InMemorySubmittedJobGraphStore implements 
SubmittedJobGraphStore {
+
+   private final Map storedJobs = new 
HashMap<>();
+
+   private boolean started;
+
+   @Override
+   public synchronized void start(@Nullable SubmittedJobGraphListener 
jobGraphListener) throws Exception {
--- End diff --

Good point.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155237910
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -86,122 +125,143 @@ public static void teardown() {
}
}
 
-   /**
-* Tests that we can submit a job to the Dispatcher which then spawns a
-* new JobManagerRunner.
-*/
-   @Test
-   public void testJobSubmission() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
+   @Before
+   public void setUp() throws Exception {
+   MockitoAnnotations.initMocks(this);
 
-   TestingLeaderElectionService dispatcherLeaderElectionService = 
new TestingLeaderElectionService();
-   TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
-   
haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService);
-   haServices.setSubmittedJobGraphStore(new 
StandaloneSubmittedJobGraphStore());
+   final JobVertex testVertex = new JobVertex("testVertex");
+   testVertex.setInvokableClass(NoOpInvokable.class);
+   jobGraph = new JobGraph(TEST_JOB_ID, "testJob", testVertex);
+   jobGraph.setAllowQueuedScheduling(true);
 
-   HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1L);
-   JobManagerRunner jobManagerRunner = 
mock(JobManagerRunner.class);
+   fatalErrorHandler = new TestingFatalErrorHandler();
+   final HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1L);
+   submittedJobGraphStore = spy(new 
InMemorySubmittedJobGraphStore());
 
-   final JobGraph jobGraph = mock(JobGraph.class);
-   final JobID jobId = new JobID();
-   when(jobGraph.getJobID()).thenReturn(jobId);
+   dispatcherLeaderElectionService = new 
TestingLeaderElectionService();
+   jobMasterLeaderElectionService = new 
TestingLeaderElectionService();
 
-   final TestingDispatcher dispatcher = new TestingDispatcher(
+   final TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
+   
haServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService);
+   haServices.setSubmittedJobGraphStore(submittedJobGraphStore);
+   haServices.setJobMasterLeaderElectionService(TEST_JOB_ID, 
jobMasterLeaderElectionService);
+   haServices.setCheckpointRecoveryFactory(new 
StandaloneCheckpointRecoveryFactory());
+   haServices.setResourceManagerLeaderRetriever(new 
TestingLeaderRetrievalService());
+   runningJobsRegistry = haServices.getRunningJobsRegistry();
+
+   final Configuration blobServerConfig = new Configuration();
+   blobServerConfig.setString(
+   BlobServerOptions.STORAGE_DIRECTORY,
+   temporaryFolder.newFolder().getAbsolutePath());
+
+   dispatcher = new TestingDispatcher(
rpcService,
Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(),
new Configuration(),
haServices,
mock(ResourceManagerGateway.class),
-   mock(BlobServer.class),
+   new BlobServer(blobServerConfig, new VoidBlobStore()),
heartbeatServices,
-   mock(MetricRegistryImpl.class),
+   new NoOpMetricRegistry(),
fatalErrorHandler,
-   jobManagerRunner,
-   jobId);
+   TEST_JOB_ID);
 
-   try {
-   dispatcher.start();
+   dispatcher.start();
+   }
 
-   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
+   @After
+   public void tearDown() throws Exception {
+   try {
+   fatalErrorHandler.rethrowError();
+   } finally {
+   RpcUtils.terminateRpcEndpoint(dispatcher, TIMEOUT);
+   }
+   }
 
-   // wait for the leader to be elected
-   leaderFuture.get();
+   /**
+* Tests that we can submit a job to the Dispatcher which then spawns a
+* new JobManagerRunner.
+*/
+   @Test
+   public void testJobSubmission() throws Exception {
+   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
 
-   DispatcherGateway 

[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155225461
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -507,6 +514,41 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   getRpcService().execute(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not recover job graph for job 
{}.", jobId, e);
+   return;
+   }
+   runAsync(() -> {
+   if (!jobManagerRunners.containsKey(jobId)) {
+   
submitJob(submittedJobGraph.getJobGraph(), RpcUtils.INF_TIMEOUT);
+   }
+   });
+   });
+   }
+
+   @Override
+   public void onRemovedJobGraph(final JobID jobId) {
+   runAsync(() -> {
+   if (jobManagerRunners.containsKey(jobId)) {
--- End diff --

This check should be superfluous because it will be check in `removeJob` as 
well.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155225963
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -86,122 +125,143 @@ public static void teardown() {
}
}
 
-   /**
-* Tests that we can submit a job to the Dispatcher which then spawns a
-* new JobManagerRunner.
-*/
-   @Test
-   public void testJobSubmission() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
+   @Before
+   public void setUp() throws Exception {
+   MockitoAnnotations.initMocks(this);
--- End diff --

No longer needed.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155229002
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/testutils/InMemorySubmittedJobGraphStore.java
 ---
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.testutils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraph;
+import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * In-Memory implementation of {@link SubmittedJobGraphStore} for testing 
purposes.
+ */
+public class InMemorySubmittedJobGraphStore implements 
SubmittedJobGraphStore {
+
+   private final Map storedJobs = new 
HashMap<>();
+
+   private boolean started;
+
+   @Override
+   public synchronized void start(@Nullable SubmittedJobGraphListener 
jobGraphListener) throws Exception {
--- End diff --

I think we should also trigger `SubmittedJobGraphListener` notifications 
when a job is added or removed from this store to make it behave similarly to 
the `ZooKeeperSubmittedJobGraphStore`.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-06 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r155225574
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -507,6 +514,41 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   getRpcService().execute(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not recover job graph for job 
{}.", jobId, e);
+   return;
+   }
+   runAsync(() -> {
+   if (!jobManagerRunners.containsKey(jobId)) {
--- End diff --

I think this check is not strictly needed.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-04 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154748301
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -244,7 +302,32 @@ protected JobManagerRunner createJobManagerRunner(
FatalErrorHandler fatalErrorHandler) throws 
Exception {
assertEquals(expectedJobId, jobGraph.getJobID());
 
-   return jobManagerRunner;
+   return new JobManagerRunner(resourceId, jobGraph, 
configuration, rpcService,
+   highAvailabilityServices, heartbeatServices, 
jobManagerServices, metricRegistry,
+   onCompleteActions, fatalErrorHandler);
+   }
+
+   @Override
+   public CompletableFuture submitJob(final JobGraph 
jobGraph, final Time timeout) {
+   final CompletableFuture submitJobFuture = 
super.submitJob(jobGraph, timeout);
+
+   try {
+   submitJobFuture.get();
+   } catch (InterruptedException e) {
+   Thread.currentThread().interrupt();
+   } catch (Exception e) {
+   throw new RuntimeException(e);
+   }
+
+   submitJobLatch.countDown();
+   return submitJobFuture;
+   }
+
+   @Override
+   void recoverJobs() {
+   if (recoverJobsEnabled.get()) {
--- End diff --

Without this I do not see how I can verify whether a job was submitted 
regularly or via `recoverJobs`.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-04 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154747386
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -534,6 +536,40 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   runAsync(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not submit job {}.", jobId, e);
--- End diff --

Changed it.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-04 Thread GJL
GitHub user GJL reopened a pull request:

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

[FLINK-8176][flip6] Start SubmittedJobGraphStore in Dispatcher

## What is the purpose of the change

The FLIP-6 dispatcher never calls `start()` on its SubmittedJobGraphStore 
instance. Hence, when a Job is submitted (YARN session mode with HA enabled), 
an IllegalStateException is thrown. This pull request adds the necessary 
changes so that jobs can be submitted.

## Brief change log

  - *Implement SubmittedJobGraphListener interface in Dispatcher*
 
## Verifying this change

  - *Added unit tests for new methods in Dispatcher class*
  - *Verified that jobs can be submitted in FLIP-6 YARN session mode with 
HA. Did not verify anything else.*


## Does this pull request potentially affect one of the following parts:

  - Dependencies (does it add or upgrade a dependency): (yes / **no**)
  - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
  - The serializers: (yes / **no** / don't know)
  - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
  - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (**yes** / no / don't know)
  - The S3 file system connector: (yes / **no** / don't know)

## Documentation

  - Does this pull request introduce a new feature? (yes / **no**)
  - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)

CC: @tillrohrmann 


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

$ git pull https://github.com/GJL/flink FLINK-8176

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

https://github.com/apache/flink/pull/5107.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #5107


commit d238ef0c23eea585974929eafdff33af916d19ba
Author: gyao 
Date:   2017-11-30T14:37:30Z

[hotfix][tests] Extract SubmittedJobGraphStore implementation from 
JobManagerHARecoveryTest

commit 33b9d2848c767088f43fed2d03e6402695827221
Author: gyao 
Date:   2017-11-30T14:44:23Z

[FLINK-8176][flip6] Implement SubmittedJobGraphListener interface in 
Dispatcher

Call start() on SubmittedJobGraphStore with Dispatcher as listener. To 
enable
this, the dispatcher must implement the SubmittedJobGraphListener 
interface. Add
simple unit tests for the new methods. Refactor DispatcherTest to remove
redundancy.

commit 88359172e23413aa195177993551613349056b68
Author: gyao 
Date:   2017-12-04T18:57:29Z

[FLINK-8176][flip6] Make InMemorySubmittedJobGraphStore thread-safe

commit 9cdb29604e9915c7d6ea60ed6fcee06c9bad57b9
Author: gyao 
Date:   2017-12-04T18:58:26Z

[hotfix][Javadoc] Make first sentence in JobSubmissionException Javadoc end 
with period

commit 53ad1771e8bec063157d69c0f7a187ccb5fb340e
Author: gyao 
Date:   2017-12-04T19:04:52Z

[FLINK-8176][flip6] Add method isStarted() to TestingLeaderElectionService

commit 0c030fb19d7b5b9dba4df5811a69086906e20ca0
Author: gyao 
Date:   2017-12-04T19:05:47Z

[FLINK-8176][flip6] Return same RunningJobsRegistry instance from 
TestingHighAvailabilityServices

commit 7a04cbe54bcf380684c4e79a4f999b31b650570e
Author: gyao 
Date:   2017-12-04T19:09:36Z

[FLINK-8176][flip6] Fix race conditions in Dispatcher and DispatcherTest

Check if jobManagerRunner exists before submitting job.
Replace JobManagerRunner mock used in tests with real instance.
Do not run job graph recovery in actor main thread when job graph is 
recovered
from SubmittedJobGraphListener#onAddedJobGraph(JobID).




---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-01 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154357254
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java 
---
@@ -534,6 +536,40 @@ public void handleError(final Exception exception) {
onFatalError(new DispatcherException("Received an error from 
the LeaderElectionService.", exception));
}
 
+   //--
+   // SubmittedJobGraphListener
+   //--
+
+   @Override
+   public void onAddedJobGraph(final JobID jobId) {
+   runAsync(() -> {
+   final SubmittedJobGraph submittedJobGraph;
+   try {
+   submittedJobGraph = 
submittedJobGraphStore.recoverJobGraph(jobId);
+   } catch (final Exception e) {
+   log.error("Could not submit job {}.", jobId, e);
--- End diff --

Maybe "could not recover job".


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-01 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154359352
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -117,84 +135,78 @@ public void testJobSubmission() throws Exception {
heartbeatServices,
mock(MetricRegistryImpl.class),
fatalErrorHandler,
-   jobManagerRunner,
-   jobId);
+   mockJobManagerRunner,
+   TEST_JOB_ID);
 
-   try {
-   dispatcher.start();
+   dispatcher.start();
+   }
 
-   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
+   @After
+   public void tearDown() throws Exception {
+   try {
+   fatalErrorHandler.rethrowError();
+   } finally {
+   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
+   }
+   }
 
-   // wait for the leader to be elected
-   leaderFuture.get();
+   /**
+* Tests that we can submit a job to the Dispatcher which then spawns a
+* new JobManagerRunner.
+*/
+   @Test
+   public void testJobSubmission() throws Exception {
+   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
 
-   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
+   // wait for the leader to be elected
+   leaderFuture.get();
 
-   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
+   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
 
-   acknowledgeFuture.get();
+   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
 
-   verify(jobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
+   acknowledgeFuture.get();
 
-   // check that no error has occurred
-   fatalErrorHandler.rethrowError();
-   } finally {
-   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
-   }
+   verify(mockJobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
}
 
/**
 * Tests that the dispatcher takes part in the leader election.
 */
@Test
public void testLeaderElection() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
-   TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
-
UUID expectedLeaderSessionId = UUID.randomUUID();
-   CompletableFuture leaderSessionIdFuture = new 
CompletableFuture<>();
-   SubmittedJobGraphStore mockSubmittedJobGraphStore = 
mock(SubmittedJobGraphStore.class);
-   TestingLeaderElectionService testingLeaderElectionService = new 
TestingLeaderElectionService() {
-   @Override
-   public void confirmLeaderSessionID(UUID 
leaderSessionId) {
-   super.confirmLeaderSessionID(leaderSessionId);
-   leaderSessionIdFuture.complete(leaderSessionId);
-   }
-   };
-
-   
haServices.setSubmittedJobGraphStore(mockSubmittedJobGraphStore);
-   
haServices.setDispatcherLeaderElectionService(testingLeaderElectionService);
-   HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1000L);
-   final JobID jobId = new JobID();
-
-   final TestingDispatcher dispatcher = new TestingDispatcher(
-   rpcService,
-   Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(),
-   new Configuration(),
-   haServices,
-   mock(ResourceManagerGateway.class),
-   mock(BlobServer.class),
-   heartbeatServices,
-   mock(MetricRegistryImpl.class),
-   fatalErrorHandler,
-   mock(JobManagerRunner.class),
-   jobId);
 
-   try {
-   dispatcher.start();
+   
assertNull(dispatcherLeaderElectionService.getConfirmationFuture());
 
-   assertFalse(leaderSessionIdFuture.isDone());
+   
dispatcherLeaderElectionService.isLeader(expectedLeaderSessionId);
 

[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-01 Thread GJL
Github user GJL closed the pull request at:

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


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-01 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154320952
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -117,84 +135,78 @@ public void testJobSubmission() throws Exception {
heartbeatServices,
mock(MetricRegistryImpl.class),
fatalErrorHandler,
-   jobManagerRunner,
-   jobId);
+   mockJobManagerRunner,
+   TEST_JOB_ID);
 
-   try {
-   dispatcher.start();
+   dispatcher.start();
+   }
 
-   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
+   @After
+   public void tearDown() throws Exception {
+   try {
+   fatalErrorHandler.rethrowError();
+   } finally {
+   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
+   }
+   }
 
-   // wait for the leader to be elected
-   leaderFuture.get();
+   /**
+* Tests that we can submit a job to the Dispatcher which then spawns a
+* new JobManagerRunner.
+*/
+   @Test
+   public void testJobSubmission() throws Exception {
+   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
 
-   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
+   // wait for the leader to be elected
+   leaderFuture.get();
 
-   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
+   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
 
-   acknowledgeFuture.get();
+   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
 
-   verify(jobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
+   acknowledgeFuture.get();
 
-   // check that no error has occurred
-   fatalErrorHandler.rethrowError();
-   } finally {
-   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
-   }
+   verify(mockJobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
}
 
/**
 * Tests that the dispatcher takes part in the leader election.
 */
@Test
public void testLeaderElection() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
-   TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
-
UUID expectedLeaderSessionId = UUID.randomUUID();
-   CompletableFuture leaderSessionIdFuture = new 
CompletableFuture<>();
-   SubmittedJobGraphStore mockSubmittedJobGraphStore = 
mock(SubmittedJobGraphStore.class);
-   TestingLeaderElectionService testingLeaderElectionService = new 
TestingLeaderElectionService() {
-   @Override
-   public void confirmLeaderSessionID(UUID 
leaderSessionId) {
-   super.confirmLeaderSessionID(leaderSessionId);
-   leaderSessionIdFuture.complete(leaderSessionId);
-   }
-   };
-
-   
haServices.setSubmittedJobGraphStore(mockSubmittedJobGraphStore);
-   
haServices.setDispatcherLeaderElectionService(testingLeaderElectionService);
-   HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1000L);
-   final JobID jobId = new JobID();
-
-   final TestingDispatcher dispatcher = new TestingDispatcher(
-   rpcService,
-   Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(),
-   new Configuration(),
-   haServices,
-   mock(ResourceManagerGateway.class),
-   mock(BlobServer.class),
-   heartbeatServices,
-   mock(MetricRegistryImpl.class),
-   fatalErrorHandler,
-   mock(JobManagerRunner.class),
-   jobId);
 
-   try {
-   dispatcher.start();
+   
assertNull(dispatcherLeaderElectionService.getConfirmationFuture());
 
-   assertFalse(leaderSessionIdFuture.isDone());
+   
dispatcherLeaderElectionService.isLeader(expectedLeaderSessionId);
 
-   

[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-12-01 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154320903
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -117,84 +135,78 @@ public void testJobSubmission() throws Exception {
heartbeatServices,
mock(MetricRegistryImpl.class),
fatalErrorHandler,
-   jobManagerRunner,
-   jobId);
+   mockJobManagerRunner,
+   TEST_JOB_ID);
 
-   try {
-   dispatcher.start();
+   dispatcher.start();
+   }
 
-   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
+   @After
+   public void tearDown() throws Exception {
+   try {
+   fatalErrorHandler.rethrowError();
+   } finally {
+   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
+   }
+   }
 
-   // wait for the leader to be elected
-   leaderFuture.get();
+   /**
+* Tests that we can submit a job to the Dispatcher which then spawns a
+* new JobManagerRunner.
+*/
+   @Test
+   public void testJobSubmission() throws Exception {
+   CompletableFuture leaderFuture = 
dispatcherLeaderElectionService.isLeader(UUID.randomUUID());
 
-   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
+   // wait for the leader to be elected
+   leaderFuture.get();
 
-   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
+   DispatcherGateway dispatcherGateway = 
dispatcher.getSelfGateway(DispatcherGateway.class);
 
-   acknowledgeFuture.get();
+   CompletableFuture acknowledgeFuture = 
dispatcherGateway.submitJob(jobGraph, timeout);
 
-   verify(jobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
+   acknowledgeFuture.get();
 
-   // check that no error has occurred
-   fatalErrorHandler.rethrowError();
-   } finally {
-   RpcUtils.terminateRpcEndpoint(dispatcher, timeout);
-   }
+   verify(mockJobManagerRunner, 
Mockito.timeout(timeout.toMilliseconds())).start();
}
 
/**
 * Tests that the dispatcher takes part in the leader election.
 */
@Test
public void testLeaderElection() throws Exception {
-   TestingFatalErrorHandler fatalErrorHandler = new 
TestingFatalErrorHandler();
-   TestingHighAvailabilityServices haServices = new 
TestingHighAvailabilityServices();
-
UUID expectedLeaderSessionId = UUID.randomUUID();
-   CompletableFuture leaderSessionIdFuture = new 
CompletableFuture<>();
-   SubmittedJobGraphStore mockSubmittedJobGraphStore = 
mock(SubmittedJobGraphStore.class);
-   TestingLeaderElectionService testingLeaderElectionService = new 
TestingLeaderElectionService() {
-   @Override
-   public void confirmLeaderSessionID(UUID 
leaderSessionId) {
-   super.confirmLeaderSessionID(leaderSessionId);
-   leaderSessionIdFuture.complete(leaderSessionId);
-   }
-   };
-
-   
haServices.setSubmittedJobGraphStore(mockSubmittedJobGraphStore);
-   
haServices.setDispatcherLeaderElectionService(testingLeaderElectionService);
-   HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1000L);
-   final JobID jobId = new JobID();
-
-   final TestingDispatcher dispatcher = new TestingDispatcher(
-   rpcService,
-   Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(),
-   new Configuration(),
-   haServices,
-   mock(ResourceManagerGateway.class),
-   mock(BlobServer.class),
-   heartbeatServices,
-   mock(MetricRegistryImpl.class),
-   fatalErrorHandler,
-   mock(JobManagerRunner.class),
-   jobId);
 
-   try {
-   dispatcher.start();
+   
assertNull(dispatcherLeaderElectionService.getConfirmationFuture());
 
-   assertFalse(leaderSessionIdFuture.isDone());
+   
dispatcherLeaderElectionService.isLeader(expectedLeaderSessionId);
 
-   

[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-11-30 Thread GJL
Github user GJL commented on a diff in the pull request:

https://github.com/apache/flink/pull/5107#discussion_r154128004
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
 ---
@@ -246,5 +258,11 @@ protected JobManagerRunner createJobManagerRunner(
 
return jobManagerRunner;
}
+
+   @Override
+   protected void runAsync(final Runnable runnable) {
--- End diff --

I think if I can set Akka's `CallingThreadDispatcher`, this is not needed.


---


[GitHub] flink pull request #5107: [FLINK-8176][flip6] Start SubmittedJobGraphStore i...

2017-11-30 Thread GJL
GitHub user GJL opened a pull request:

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

[FLINK-8176][flip6] Start SubmittedJobGraphStore in Dispatcher

## What is the purpose of the change

The FLIP-6 dispatcher never calls `start()` on its SubmittedJobGraphStore 
instance. Hence, when a Job is submitted (YARN session mode with HA enabled), 
an IllegalStateException is thrown. This pull request adds the necessary 
changes so that jobs can be submitted.

## Brief change log

  - *Implement SubmittedJobGraphListener interface in Dispatcher*
 
## Verifying this change

  - *Added unit tests for new methods in Dispatcher class*
  - *Verified that jobs can be submitted in FLIP-6 YARN session mode with 
HA. Did not verify anything else.*


## Does this pull request potentially affect one of the following parts:

  - Dependencies (does it add or upgrade a dependency): (yes / **no**)
  - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (yes / **no**)
  - The serializers: (yes / **no** / don't know)
  - The runtime per-record code paths (performance sensitive): (yes / 
**no** / don't know)
  - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (**yes** / no / don't know)
  - The S3 file system connector: (yes / **no** / don't know)

## Documentation

  - Does this pull request introduce a new feature? (yes / **no**)
  - If yes, how is the feature documented? (**not applicable** / docs / 
JavaDocs / not documented)

CC: @tillrohrmann 


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

$ git pull https://github.com/GJL/flink FLINK-8176

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

https://github.com/apache/flink/pull/5107.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #5107


commit d238ef0c23eea585974929eafdff33af916d19ba
Author: gyao 
Date:   2017-11-30T14:37:30Z

[hotfix][tests] Extract SubmittedJobGraphStore implementation from 
JobManagerHARecoveryTest

commit 33b9d2848c767088f43fed2d03e6402695827221
Author: gyao 
Date:   2017-11-30T14:44:23Z

[FLINK-8176][flip6] Implement SubmittedJobGraphListener interface in 
Dispatcher

Call start() on SubmittedJobGraphStore with Dispatcher as listener. To 
enable
this, the dispatcher must implement the SubmittedJobGraphListener 
interface. Add
simple unit tests for the new methods. Refactor DispatcherTest to remove
redundancy.




---