From 8daa03bc731fdaad83f4bc39e8d9356cf3f37bda Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 30 Jan 2018 15:15:49 +0100 Subject: [PATCH 1/8] [FLINK-8529] [flip6] Let Yarn entry points use APPLICATION_MASTER_PORT Let all Yarn entry points use the YarnConfigOptions.APPLICATION_MASTER_PORT option to specify the valid port range for the common RpcService. This closes #5388. --- .../runtime/entrypoint/ClusterEntrypoint.java | 18 ++++++++++++++++-- .../entrypoint/YarnJobClusterEntrypoint.java | 6 ++++++ .../YarnSessionClusterEntrypoint.java | 6 ++++++ 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 156efdc5beccf..dfb1b4ce9e2b1 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -172,10 +172,14 @@ protected void initializeServices(Configuration configuration) throws Exception LOG.info("Initializing cluster services."); final String bindAddress = configuration.getString(JobManagerOptions.ADDRESS); - // TODO: Add support for port ranges - final String portRange = String.valueOf(configuration.getInteger(JobManagerOptions.PORT)); + final String portRange = getRPCPortRange(configuration); commonRpcService = createRpcService(configuration, bindAddress, portRange); + + // update the configuration used to create the high availability services + configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); + configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort()); + haServices = createHaServices(configuration, commonRpcService.getExecutor()); blobServer = new BlobServer(configuration, haServices.createBlobStore()); blobServer.start(); @@ -188,6 +192,16 @@ protected void initializeServices(Configuration configuration) throws Exception metricRegistry.startQueryService(actorSystem, null); } + /** + * Returns the port range for the common {@link RpcService}. + * + * @param configuration to extract the port range from + * @return Port range for the common {@link RpcService} + */ + protected String getRPCPortRange(Configuration configuration) { + return String.valueOf(configuration.getInteger(JobManagerOptions.PORT)); + } + protected RpcService createRpcService( Configuration configuration, String bindAddress, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java index 4a9442a71807e..ca4cb9cddf422 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnJobClusterEntrypoint.java @@ -39,6 +39,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.YarnResourceManager; +import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.hadoop.yarn.api.ApplicationConstants; @@ -74,6 +75,11 @@ protected SecurityContext installSecurityContext(Configuration configuration) th return YarnEntrypointUtils.installSecurityContext(configuration, workingDirectory); } + @Override + protected String getRPCPortRange(Configuration configuration) { + return configuration.getString(YarnConfigOptions.APPLICATION_MASTER_PORT); + } + @Override protected ResourceManager createResourceManager( Configuration configuration, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java index 64b4aca4553e0..232467a640357 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnSessionClusterEntrypoint.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.util.SignalHandler; import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.YarnResourceManager; +import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.hadoop.yarn.api.ApplicationConstants; @@ -64,6 +65,11 @@ protected SecurityContext installSecurityContext(Configuration configuration) th return YarnEntrypointUtils.installSecurityContext(configuration, workingDirectory); } + @Override + protected String getRPCPortRange(Configuration configuration) { + return configuration.getString(YarnConfigOptions.APPLICATION_MASTER_PORT); + } + @Override protected ResourceManager createResourceManager( Configuration configuration, From bacb438b6c8e002375d1d1753dbbc357fbff91d0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 6 Feb 2018 16:46:52 +0100 Subject: [PATCH 2/8] [FLINK-8603] [flip6] Split submitJob into job submission and execution result retrieval Split RestClusterClient#submitJob into submitJob and requestJobResult which can be called individually. This closes #5428. --- .../program/rest/RestClusterClient.java | 133 +++++++++++------- 1 file changed, 83 insertions(+), 50 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 564990fdf6349..18cd5c0a4cd49 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -96,6 +96,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -198,31 +199,22 @@ public void shutdown() { protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { log.info("Submitting job."); try { - // we have to enable queued scheduling because slot will be allocated lazily - jobGraph.setAllowQueuedScheduling(true); - submitJob(jobGraph); - } catch (JobSubmissionException e) { - throw new ProgramInvocationException(e); + submitJob(jobGraph).get(); + } catch (InterruptedException | ExecutionException e) { + throw new ProgramInvocationException(ExceptionUtils.stripExecutionException(e)); } - final JobResult jobExecutionResult; + final CompletableFuture jobResultFuture = requestJobResult(jobGraph.getJobID()); + + final JobResult jobResult; try { - jobExecutionResult = pollResourceAsync( - () -> { - final JobMessageParameters messageParameters = new JobMessageParameters(); - messageParameters.jobPathParameter.resolve(jobGraph.getJobID()); - return sendRetryableRequest( - JobExecutionResultHeaders.getInstance(), - messageParameters, - EmptyRequestBody.getInstance(), - isConnectionProblemException().or(isHttpStatusUnsuccessfulException())); - }).get(); - } catch (final Exception e) { + jobResult = jobResultFuture.get(); + } catch (Exception e) { throw new ProgramInvocationException(e); } - if (jobExecutionResult.getSerializedThrowable().isPresent()) { - final SerializedThrowable serializedThrowable = jobExecutionResult.getSerializedThrowable().get(); + if (jobResult.getSerializedThrowable().isPresent()) { + final SerializedThrowable serializedThrowable = jobResult.getSerializedThrowable().get(); final Throwable throwable = serializedThrowable.deserializeError(classLoader); throw new ProgramInvocationException(throwable); } @@ -231,10 +223,10 @@ protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoad // don't return just a JobSubmissionResult here, the signature is lying // The CliFrontend expects this to be a JobExecutionResult this.lastJobExecutionResult = new JobExecutionResult( - jobExecutionResult.getJobId(), - jobExecutionResult.getNetRuntime(), + jobResult.getJobId(), + jobResult.getNetRuntime(), AccumulatorHelper.deserializeAccumulators( - jobExecutionResult.getAccumulatorResults(), + jobResult.getAccumulatorResults(), classLoader)); return lastJobExecutionResult; } catch (IOException | ClassNotFoundException e) { @@ -242,37 +234,78 @@ protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoad } } - private void submitJob(JobGraph jobGraph) throws JobSubmissionException { + /** + * Requests the {@link JobResult} for the given {@link JobID}. The method retries multiple + * times to poll the {@link JobResult} before giving up. + * + * @param jobId specifying the job for which to retrieve the {@link JobResult} + * @return Future which is completed with the {@link JobResult} once the job has completed or + * with a failure if the {@link JobResult} could not be retrieved. + */ + public CompletableFuture requestJobResult(JobID jobId) { + return pollResourceAsync( + () -> { + final JobMessageParameters messageParameters = new JobMessageParameters(); + messageParameters.jobPathParameter.resolve(jobId); + return sendRetryableRequest( + JobExecutionResultHeaders.getInstance(), + messageParameters, + EmptyRequestBody.getInstance(), + isConnectionProblemException().or(isHttpStatusUnsuccessfulException())); + }); + } + + /** + * Submits the given {@link JobGraph} to the dispatcher. + * + * @param jobGraph to submit + * @return Future which is completed with the submission response + */ + public CompletableFuture submitJob(JobGraph jobGraph) { + // we have to enable queued scheduling because slot will be allocated lazily + jobGraph.setAllowQueuedScheduling(true); + log.info("Requesting blob server port."); - int blobServerPort; - try { - CompletableFuture portFuture = sendRequest( - BlobServerPortHeaders.getInstance()); - blobServerPort = portFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS).port; - } catch (Exception e) { - throw new JobSubmissionException(jobGraph.getJobID(), "Failed to retrieve blob server port.", e); - } + CompletableFuture portFuture = sendRequest( + BlobServerPortHeaders.getInstance()); + + CompletableFuture jobUploadFuture = portFuture.thenCombine( + getDispatcherAddress(), + (BlobServerPortResponseBody response, String dispatcherAddress) -> { + log.info("Uploading jar files."); + final int blobServerPort = response.port; + final InetSocketAddress address = new InetSocketAddress(dispatcherAddress, blobServerPort); + final List keys; + try { + keys = BlobClient.uploadJarFiles(address, flinkConfig, jobGraph.getJobID(), jobGraph.getUserJars()); + } catch (IOException ioe) { + throw new CompletionException(new FlinkException("Could not upload job jar files.", ioe)); + } - log.info("Uploading jar files."); - try { - InetSocketAddress address = new InetSocketAddress(getDispatcherAddress().get(), blobServerPort); - List keys = BlobClient.uploadJarFiles(address, this.flinkConfig, jobGraph.getJobID(), jobGraph.getUserJars()); - for (PermanentBlobKey key : keys) { - jobGraph.addBlob(key); - } - } catch (Exception e) { - throw new JobSubmissionException(jobGraph.getJobID(), "Failed to upload user jars to blob server.", e); - } + for (PermanentBlobKey key : keys) { + jobGraph.addBlob(key); + } - log.info("Submitting job graph."); - try { - CompletableFuture responseFuture = sendRequest( - JobSubmitHeaders.getInstance(), - new JobSubmitRequestBody(jobGraph)); - responseFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS); - } catch (Exception e) { - throw new JobSubmissionException(jobGraph.getJobID(), "Failed to submit JobGraph.", e); - } + return jobGraph; + }); + + CompletableFuture submissionFuture = jobUploadFuture.thenCompose( + (JobGraph jobGraphToSubmit) -> { + log.info("Submitting job graph."); + + try { + return sendRequest( + JobSubmitHeaders.getInstance(), + new JobSubmitRequestBody(jobGraph)); + } catch (IOException ioe) { + throw new CompletionException(new FlinkException("Could not create JobSubmitRequestBody.", ioe)); + } + }); + + return submissionFuture.exceptionally( + (Throwable throwable) -> { + throw new CompletionException(new JobSubmissionException(jobGraph.getJobID(), "Failed to submit JobGraph.", throwable)); + }); } @Override From 8bf78ee4512e852b401568fada858f31f00d475e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 6 Feb 2018 19:11:53 +0100 Subject: [PATCH 3/8] [hotfix] Change shutdown order in WebMonitorEndpoint to avoid illegal state --- .../apache/flink/runtime/webmonitor/WebMonitorEndpoint.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 021a009cd62f2..dfcb1341303e3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -575,8 +575,6 @@ public void start() throws Exception { @Override public void shutdown(Time timeout) { - super.shutdown(timeout); - executionGraphCache.close(); final File tmpDir = restConfiguration.getTmpDir(); @@ -593,6 +591,8 @@ public void shutdown(Time timeout) { } catch (Exception e) { log.warn("Error while stopping leaderElectionService", e); } + + super.shutdown(timeout); } //------------------------------------------------------------------------- From da5f4ce7f7cab0a0f2fc59c4890e6363abf51252 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 8 Feb 2018 13:43:09 +0100 Subject: [PATCH 4/8] [FLINK-8604] [rest] Move JobTerminationHandler into WebMonitorEndpoint Register the JobTerminationHandler at the WebMonitorEndpoint to make it accessible to all REST endpoints. This closes #5429. --- .../runtime/dispatcher/DispatcherGateway.java | 18 ------ .../dispatcher/DispatcherRestEndpoint.java | 10 ---- .../flink/runtime/jobmaster/JobMaster.java | 18 ++++++ .../handler/job/JobTerminationHandler.java | 8 +-- .../runtime/webmonitor/RestfulGateway.java | 19 ++++++ .../webmonitor/WebMonitorEndpoint.java | 10 ++++ .../job/JobVertexBackPressureHandlerTest.java | 2 +- .../webmonitor/TestingRestfulGateway.java | 59 +++++++++++++++++-- 8 files changed, 106 insertions(+), 38 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java index 967642921f8e5..5f5ad1c3ac45d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java @@ -56,24 +56,6 @@ CompletableFuture submitJob( CompletableFuture> listJobs( @RpcTimeout Time timeout); - /** - * Cancel the given job. - * - * @param jobId identifying the job to cancel - * @param timeout of the operation - * @return A future acknowledge if the cancellation succeeded - */ - CompletableFuture cancelJob(JobID jobId, @RpcTimeout Time timeout); - - /** - * Stop the given job. - * - * @param jobId identifying the job to stop - * @param timeout of the operation - * @return A future acknowledge if the stopping succeeded - */ - CompletableFuture stopJob(JobID jobId, @RpcTimeout Time timeout); - /** * Returns the port of the blob server. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index f9263c954e65b..90022ed37ed03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -29,8 +29,6 @@ import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.handler.job.BlobServerPortHandler; import org.apache.flink.runtime.rest.handler.job.JobSubmitHandler; -import org.apache.flink.runtime.rest.handler.job.JobTerminationHandler; -import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; @@ -81,13 +79,6 @@ protected List> initiali final Time timeout = restConfiguration.getTimeout(); final Map responseHeaders = restConfiguration.getResponseHeaders(); - JobTerminationHandler jobTerminationHandler = new JobTerminationHandler( - restAddressFuture, - leaderRetriever, - timeout, - responseHeaders, - JobTerminationHeaders.getInstance()); - BlobServerPortHandler blobServerPortHandler = new BlobServerPortHandler( restAddressFuture, leaderRetriever, @@ -100,7 +91,6 @@ protected List> initiali timeout, responseHeaders); - handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobTerminationHandler)); handlers.add(Tuple2.of(blobServerPortHandler.getMessageHeaders(), blobServerPortHandler)); handlers.add(Tuple2.of(jobSubmitHandler.getMessageHeaders(), jobSubmitHandler)); 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 bf24dc69f0fb7..e7583d007320f 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 @@ -822,6 +822,24 @@ public CompletableFuture requestJobStatus(Time timeout) { // RestfulGateway RPC methods //---------------------------------------------------------------------------------------------- + @Override + public CompletableFuture cancelJob(JobID jobId, Time timeout) { + if (jobGraph.getJobID().equals(jobId)) { + return cancel(timeout); + } else { + return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); + } + } + + @Override + public CompletableFuture stopJob(JobID jobId, Time timeout) { + if (jobGraph.getJobID().equals(jobId)) { + return stop(timeout); + } else { + return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); + } + } + @Override public CompletableFuture requestRestAddress(Time timeout) { return restAddressFuture; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobTerminationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobTerminationHandler.java index 0998177816159..96f57f81d8c99 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobTerminationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobTerminationHandler.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.rest.handler.AbstractRestHandler; @@ -33,6 +32,7 @@ import org.apache.flink.runtime.rest.messages.JobTerminationMessageParameters; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter; +import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.ExceptionUtils; @@ -47,11 +47,11 @@ /** * Request handler for the cancel and stop request. */ -public class JobTerminationHandler extends AbstractRestHandler { +public class JobTerminationHandler extends AbstractRestHandler { public JobTerminationHandler( CompletableFuture localRestAddress, - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, Time timeout, Map headers, MessageHeaders messageHeaders) { @@ -59,7 +59,7 @@ public JobTerminationHandler( } @Override - public CompletableFuture handleRequest(HandlerRequest request, DispatcherGateway gateway) { + public CompletableFuture handleRequest(HandlerRequest request, RestfulGateway gateway) { final JobID jobId = request.getPathParameter(JobIDPathParameter.class); final List terminationModes = request.getQueryParameter(TerminationModeQueryParameter.class); final TerminationModeQueryParameter.TerminationMode terminationMode; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java index 28465a73615b3..6cad0fb39a45c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; @@ -46,6 +47,24 @@ */ public interface RestfulGateway extends RpcGateway { + /** + * Cancel the given job. + * + * @param jobId identifying the job to cancel + * @param timeout of the operation + * @return A future acknowledge if the cancellation succeeded + */ + CompletableFuture cancelJob(JobID jobId, @RpcTimeout Time timeout); + + /** + * Stop the given job. + * + * @param jobId identifying the job to stop + * @param timeout of the operation + * @return A future acknowledge if the stopping succeeded + */ + CompletableFuture stopJob(JobID jobId, @RpcTimeout Time timeout); + /** * Requests the REST address of this {@link RpcEndpoint}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index dfcb1341303e3..5aa3271973c79 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.rest.handler.job.JobExecutionResultHandler; import org.apache.flink.runtime.rest.handler.job.JobIdsHandler; import org.apache.flink.runtime.rest.handler.job.JobPlanHandler; +import org.apache.flink.runtime.rest.handler.job.JobTerminationHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexAccumulatorsHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexBackPressureHandler; import org.apache.flink.runtime.rest.handler.job.JobVertexTaskManagersHandler; @@ -78,6 +79,7 @@ import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; import org.apache.flink.runtime.rest.messages.JobIdsWithStatusesOverviewHeaders; import org.apache.flink.runtime.rest.messages.JobPlanHeaders; +import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; import org.apache.flink.runtime.rest.messages.JobVertexAccumulatorsHeaders; import org.apache.flink.runtime.rest.messages.JobVertexBackPressureHeaders; import org.apache.flink.runtime.rest.messages.JobVertexTaskManagersHeaders; @@ -449,6 +451,13 @@ protected List> initiali responseHeaders, JobVertexBackPressureHeaders.getInstance()); + final JobTerminationHandler jobTerminationHandler = new JobTerminationHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobTerminationHeaders.getInstance()); + final File tmpDir = restConfiguration.getTmpDir(); Optional> optWebContent; @@ -495,6 +504,7 @@ protected List> initiali handlers.add(Tuple2.of(SubtaskCurrentAttemptDetailsHeaders.getInstance(), subtaskCurrentAttemptDetailsHandler)); handlers.add(Tuple2.of(JobVertexTaskManagersHeaders.getInstance(), jobVertexTaskManagersHandler)); handlers.add(Tuple2.of(JobVertexBackPressureHeaders.getInstance(), jobVertexBackPressureHandler)); + handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobTerminationHandler)); optWebContent.ifPresent( webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java index b5a6ba67d25d4..6e97f6b394c5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobVertexBackPressureHandlerTest.java @@ -70,7 +70,7 @@ public class JobVertexBackPressureHandlerTest { @Before public void setUp() { - restfulGateway = TestingRestfulGateway.newBuilder().setRequestOeratorBackPressureStatsFunction( + restfulGateway = TestingRestfulGateway.newBuilder().setRequestOperatorBackPressureStatsFunction( (jobId, jobVertexId) -> { if (jobId.equals(TEST_JOB_ID_BACK_PRESSURE_STATS_AVAILABLE)) { return CompletableFuture.completedFuture(OperatorBackPressureStatsResponse.of(new OperatorBackPressureStats( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java index 8b9f671f90650..dc88f7350b2d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.webmonitor.ClusterOverview; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; @@ -42,6 +43,8 @@ */ public class TestingRestfulGateway implements RestfulGateway { + static final Function> DEFAULT_CANCEL_JOB_FUNCTION = jobId -> CompletableFuture.completedFuture(Acknowledge.get()); + static final Function> DEFAULT_STOP_JOB_FUNCTION = jobId -> CompletableFuture.completedFuture(Acknowledge.get()); static final Function> DEFAULT_REQUEST_JOB_FUNCTION = jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); static final Function> DEFAULT_REQUEST_JOB_STATUS_FUNCTION = jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException()); static final Supplier> DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER = () -> CompletableFuture.completedFuture(new MultipleJobsDetails(Collections.emptyList())); @@ -57,6 +60,10 @@ public class TestingRestfulGateway implements RestfulGateway { protected String restAddress; + protected Function> cancelJobFunction; + + protected Function> stopJobFunction; + protected Function> requestJobFunction; protected Function> requestJobStatusFunction; @@ -76,6 +83,8 @@ public TestingRestfulGateway() { LOCALHOST, LOCALHOST, LOCALHOST, + DEFAULT_CANCEL_JOB_FUNCTION, + DEFAULT_STOP_JOB_FUNCTION, DEFAULT_REQUEST_JOB_FUNCTION, DEFAULT_REQUEST_JOB_STATUS_FUNCTION, DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER, @@ -89,6 +98,8 @@ public TestingRestfulGateway( String address, String hostname, String restAddress, + Function> cancelJobFunction, + Function> stopJobFunction, Function> requestJobFunction, Function> requestJobStatusFunction, Supplier> requestMultipleJobDetailsSupplier, @@ -99,6 +110,8 @@ public TestingRestfulGateway( this.address = address; this.hostname = hostname; this.restAddress = restAddress; + this.cancelJobFunction = cancelJobFunction; + this.stopJobFunction = stopJobFunction; this.requestJobFunction = requestJobFunction; this.requestJobStatusFunction = requestJobStatusFunction; this.requestMultipleJobDetailsSupplier = requestMultipleJobDetailsSupplier; @@ -108,6 +121,16 @@ public TestingRestfulGateway( this.requestOperatorBackPressureStatsFunction = requestOperatorBackPressureStatsFunction; } + @Override + public CompletableFuture cancelJob(JobID jobId, Time timeout) { + return cancelJobFunction.apply(jobId); + } + + @Override + public CompletableFuture stopJob(JobID jobId, Time timeout) { + return stopJobFunction.apply(jobId); + } + @Override public CompletableFuture requestRestAddress(Time timeout) { return CompletableFuture.completedFuture(restAddress); @@ -169,22 +192,26 @@ public static final class Builder { private String address = LOCALHOST; private String hostname = LOCALHOST; private String restAddress = LOCALHOST; + private Function> cancelJobFunction; + private Function> stopJobFunction; private Function> requestJobFunction; private Function> requestJobStatusFunction; private Supplier> requestMultipleJobDetailsSupplier; private Supplier> requestClusterOverviewSupplier; private Supplier>> requestMetricQueryServicePathsSupplier; private Supplier>>> requestTaskManagerMetricQueryServicePathsSupplier; - private BiFunction> requestOeratorBackPressureStatsFunction; + private BiFunction> requestOperatorBackPressureStatsFunction; public Builder() { + cancelJobFunction = DEFAULT_CANCEL_JOB_FUNCTION; + stopJobFunction = DEFAULT_STOP_JOB_FUNCTION; requestJobFunction = DEFAULT_REQUEST_JOB_FUNCTION; requestJobStatusFunction = DEFAULT_REQUEST_JOB_STATUS_FUNCTION; requestMultipleJobDetailsSupplier = DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER; requestClusterOverviewSupplier = DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER; requestMetricQueryServicePathsSupplier = DEFAULT_REQUEST_METRIC_QUERY_SERVICE_PATHS_SUPPLIER; requestTaskManagerMetricQueryServicePathsSupplier = DEFAULT_REQUEST_TASK_MANAGER_METRIC_QUERY_SERVICE_PATHS_SUPPLIER; - requestOeratorBackPressureStatsFunction = DEFAULT_REQUEST_OPERATOR_BACK_PRESSURE_STATS_SUPPLIER; + requestOperatorBackPressureStatsFunction = DEFAULT_REQUEST_OPERATOR_BACK_PRESSURE_STATS_SUPPLIER; } public Builder setAddress(String address) { @@ -232,13 +259,35 @@ public Builder setRequestTaskManagerMetricQueryServicePathsSupplier(Supplier> requestOeratorBackPressureStatsFunction) { - this.requestOeratorBackPressureStatsFunction = requestOeratorBackPressureStatsFunction; + public Builder setRequestOperatorBackPressureStatsFunction(BiFunction> requestOeratorBackPressureStatsFunction) { + this.requestOperatorBackPressureStatsFunction = requestOeratorBackPressureStatsFunction; + return this; + } + + public Builder setCancelJobFunction(Function> cancelJobFunction) { + this.cancelJobFunction = cancelJobFunction; + return this; + } + + public Builder setStopJobFunction(Function> stopJobFunction) { + this.stopJobFunction = stopJobFunction; return this; } public TestingRestfulGateway build() { - return new TestingRestfulGateway(address, hostname, restAddress, requestJobFunction, requestJobStatusFunction, requestMultipleJobDetailsSupplier, requestClusterOverviewSupplier, requestMetricQueryServicePathsSupplier, requestTaskManagerMetricQueryServicePathsSupplier, requestOeratorBackPressureStatsFunction); + return new TestingRestfulGateway( + address, + hostname, + restAddress, + cancelJobFunction, + stopJobFunction, + requestJobFunction, + requestJobStatusFunction, + requestMultipleJobDetailsSupplier, + requestClusterOverviewSupplier, + requestMetricQueryServicePathsSupplier, + requestTaskManagerMetricQueryServicePathsSupplier, + requestOperatorBackPressureStatsFunction); } } } From 56e4d587b76f5fa871492bcea3249584ec16aec6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Feb 2018 12:35:58 +0100 Subject: [PATCH 5/8] [FLINK-8605] [rest] Enable job cancellation from the web UI In order to support the job cancellation from the web UI, including when using Yarn, we have to register the JobTerminationHandler under /jobs/:jobid/yarn-cancel and /jobs/:jobid/yarn-stop. This is just a temporary fix until we can send arbitrary REST verbs through the Yarn proxy. This closes #5430. --- .../app/scripts/modules/jobs/jobs.svc.coffee | 4 +- .../web-dashboard/web/js/hs/index.js | 4 +- .../web-dashboard/web/js/index.js | 4 +- .../handler/job/JobTerminationHandler.java | 10 +++- .../YarnCancelJobTerminationHeaders.java | 55 +++++++++++++++++++ .../YarnStopJobTerminationHeaders.java | 55 +++++++++++++++++++ .../webmonitor/WebMonitorEndpoint.java | 22 +++++++- .../YarnCancelJobTerminationHeadersTest.java | 47 ++++++++++++++++ .../YarnStopJobTerminationHeadersTest.java | 48 ++++++++++++++++ 9 files changed, 238 insertions(+), 11 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/YarnCancelJobTerminationHeaders.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeaders.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnCancelJobTerminationHeadersTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeadersTest.java diff --git a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee index 5441730d331df..b5554303551a8 100644 --- a/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee +++ b/flink-runtime-web/web-dashboard/app/scripts/modules/jobs/jobs.svc.coffee @@ -314,12 +314,12 @@ angular.module('flinkApp') @cancelJob = (jobid) -> # uses the non REST-compliant GET yarn-cancel handler which is available in addition to the - # proper "DELETE jobs//" + # proper $http.patch flinkConfig.jobServer + "jobs/" + jobid + "?mode=cancel" $http.get flinkConfig.jobServer + "jobs/" + jobid + "/yarn-cancel" @stopJob = (jobid) -> # uses the non REST-compliant GET yarn-cancel handler which is available in addition to the - # proper "DELETE jobs//" + # proper $http.patch flinkConfig.jobServer + "jobs/" + jobid + "?mode=stop" $http.get "jobs/" + jobid + "/yarn-stop" @ diff --git a/flink-runtime-web/web-dashboard/web/js/hs/index.js b/flink-runtime-web/web-dashboard/web/js/hs/index.js index 72e1d26ed38c0..7cd45e19e3366 100644 --- a/flink-runtime-web/web-dashboard/web/js/hs/index.js +++ b/flink-runtime-web/web-dashboard/web/js/hs/index.js @@ -1,2 +1,2 @@ -angular.module("flinkApp",["ui.router","angularMoment","dndLists"]).run(["$rootScope",function(e){return e.sidebarVisible=!1,e.showSidebar=function(){return e.sidebarVisible=!e.sidebarVisible,e.sidebarClass="force-show"}}]).value("flinkConfig",{jobServer:"","refresh-interval":1e4}).value("watermarksConfig",{noWatermark:-0x8000000000000000}).run(["JobsService","MainService","flinkConfig","$interval",function(e,t,r,n){return t.loadConfig().then(function(t){return angular.extend(r,t),e.listJobs(),n(function(){return e.listJobs()},r["refresh-interval"])})}]).config(["$uiViewScrollProvider",function(e){return e.useAnchorScroll()}]).run(["$rootScope","$state",function(e,t){return e.$on("$stateChangeStart",function(e,r,n,i){if(r.redirectTo)return e.preventDefault(),t.go(r.redirectTo,n)})}]).config(["$stateProvider","$urlRouterProvider",function(e,t){return e.state("completed-jobs",{url:"/completed-jobs",views:{main:{templateUrl:"partials/jobs/completed-jobs.html",controller:"CompletedJobsController"}}}).state("single-job",{url:"/jobs/{jobid}","abstract":!0,views:{main:{templateUrl:"partials/jobs/job.html",controller:"SingleJobController"}}}).state("single-job.plan",{url:"",redirectTo:"single-job.plan.subtasks",views:{details:{templateUrl:"partials/jobs/job.plan.html",controller:"JobPlanController"}}}).state("single-job.plan.subtasks",{url:"",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.subtasks.html",controller:"JobPlanSubtasksController"}}}).state("single-job.plan.metrics",{url:"/metrics",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.metrics.html",controller:"JobPlanMetricsController"}}}).state("single-job.plan.watermarks",{url:"/watermarks",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.watermarks.html"}}}).state("single-job.plan.taskmanagers",{url:"/taskmanagers",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.taskmanagers.html",controller:"JobPlanTaskManagersController"}}}).state("single-job.plan.accumulators",{url:"/accumulators",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.accumulators.html",controller:"JobPlanAccumulatorsController"}}}).state("single-job.plan.checkpoints",{url:"/checkpoints",redirectTo:"single-job.plan.checkpoints.overview",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.checkpoints.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.overview",{url:"/overview",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.overview.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.summary",{url:"/summary",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.summary.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.history",{url:"/history",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.history.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.config",{url:"/config",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.config.html",controller:"JobPlanCheckpointsController"}}}).state("single-job.plan.checkpoints.details",{url:"/details/{checkpointId}",views:{"checkpoints-view":{templateUrl:"partials/jobs/job.plan.node.checkpoints.details.html",controller:"JobPlanCheckpointDetailsController"}}}).state("single-job.plan.backpressure",{url:"/backpressure",views:{"node-details":{templateUrl:"partials/jobs/job.plan.node-list.backpressure.html",controller:"JobPlanBackPressureController"}}}).state("single-job.timeline",{url:"/timeline",views:{details:{templateUrl:"partials/jobs/job.timeline.html"}}}).state("single-job.timeline.vertex",{url:"/{vertexId}",views:{vertex:{templateUrl:"partials/jobs/job.timeline.vertex.html",controller:"JobTimelineVertexController"}}}).state("single-job.exceptions",{url:"/exceptions",views:{details:{templateUrl:"partials/jobs/job.exceptions.html",controller:"JobExceptionsController"}}}).state("single-job.config",{url:"/config",views:{details:{templateUrl:"partials/jobs/job.config.html"}}}),t.otherwise("/completed-jobs")}]),angular.module("flinkApp").directive("bsLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentLowWatermark",""),l[s]=a,(isNaN(i)||au.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=J(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+J(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null, -callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,s,a;return"undefined"==typeof e||null===e?"":(o=e%1e3,a=Math.floor(e/1e3),s=a%60,a=Math.floor(a/60),i=a%60,a=Math.floor(a/60),n=a%24,a=Math.floor(a/24),r=a,0===r?0===n?0===i?0===s?o+"ms":s+"s ":i+"m "+s+"s":t?n+"h "+i+"m":n+"h "+i+"m "+s+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+s+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,s).then(function(e){var t,n,i,o,s,a,l;i=NaN,l={},o=e.values;for(t in o)a=o[t],s=t.replace(".currentLowWatermark",""),l[s]=a,(isNaN(i)||au.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),s={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(s[i]=e,t>=n-1)return r.resolve(s)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,s,a){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,s,a;s=r.children()[0],o=r.width(),angular.element(s).attr("width",o),a=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(s).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:a(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:a(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(s).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,s,a,l,u,c,d,f,p,m,h,g,b,v,k,j,S,w,C,$,y,J,M;p=null,C=d3.behavior.zoom(),M=[],g=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},h=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":h(t)?"node-iteration":"node-normal"},s=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=J(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(h(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+J(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},J=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},a=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:s(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:s(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:s(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:s(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:s(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:s(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:s(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,s,l,u,d,f,p,m,h,g,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),s=0,u=v.length;s-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return s=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,a,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),s=i,l.job.resolve(s)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,s;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){ +return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,s){if(o.id===r.id&&(i.metrics[e][t].splice(s,1),s",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||su.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(a[i]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,a,s){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(a).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(a).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,M,y,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=y(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(g(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+y(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},y=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),a=0,u=v.length;a-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e)); -},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,a){if(o.id===r.id&&(i.metrics[e][t].splice(a,1),a",link:function(t,r,n){return t.getLabelClass=function(){return"label label-"+e.translateLabelState(n.status)}}}}]).directive("bpLabel",["JobsService",function(e){return{transclude:!0,replace:!0,scope:{getBackPressureLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getBackPressureLabelClass=function(){return"label label-"+e.translateBackPressureLabelState(n.status)}}}}]).directive("indicatorPrimary",["JobsService",function(e){return{replace:!0,scope:{getLabelClass:"&",status:"@"},template:"",link:function(t,r,n){return t.getLabelClass=function(){return"fa fa-circle indicator indicator-"+e.translateLabelState(n.status)}}}}]).directive("tableProperty",function(){return{replace:!0,scope:{value:"="},template:"{{value || 'None'}}"}}),angular.module("flinkApp").filter("amDurationFormatExtended",["angularMomentConfig",function(e){var t;return t=function(e,t,r){return"undefined"==typeof e||null===e?"":moment.duration(e,t).format(r,{trim:!1})},t.$stateful=e.statefulFilters,t}]).filter("humanizeDuration",function(){return function(e,t){var r,n,i,o,a,s;return"undefined"==typeof e||null===e?"":(o=e%1e3,s=Math.floor(e/1e3),a=s%60,s=Math.floor(s/60),i=s%60,s=Math.floor(s/60),n=s%24,s=Math.floor(s/24),r=s,0===r?0===n?0===i?0===a?o+"ms":a+"s ":i+"m "+a+"s":t?n+"h "+i+"m":n+"h "+i+"m "+a+"s":t?r+"d "+n+"h":r+"d "+n+"h "+i+"m "+a+"s")}}).filter("limit",function(){return function(e){return e.length>73&&(e=e.substring(0,35)+"..."+e.substring(e.length-35,e.length)),e}}).filter("humanizeText",function(){return function(e){return e?e.replace(/>/g,">").replace(//g,""):""}}).filter("humanizeBytes",function(){return function(e){var t,r;return r=["B","KB","MB","GB","TB","PB","EB"],t=function(e,n){var i;return i=Math.pow(1024,n),e=r;n=0<=r?++e:--e)i.push(n+".currentLowWatermark");return i}(),i.getMetrics(o,t.id,a).then(function(e){var t,n,i,o,a,s,l;i=NaN,l={},o=e.values;for(t in o)s=o[t],a=t.replace(".currentLowWatermark",""),l[a]=s,(isNaN(i)||su.noWatermark?i:NaN,r.resolve({lowWatermark:n,watermarks:l})}),r.promise}}(this),r=l.defer(),a={},n=t.length,angular.forEach(t,function(e){return function(e,t){var i;return i=e.id,o(e).then(function(e){if(a[i]=e,t>=n-1)return r.resolve(a)})}}(this)),r.promise},e.hasWatermark=function(t){return e.watermarks[t]&&!isNaN(e.watermarks[t].lowWatermark)},e.$watch("plan",function(t){if(t)return c(t.nodes).then(function(t){return e.watermarks=t})}),e.$on("reload",function(){if(e.plan)return c(e.plan.nodes).then(function(t){return e.watermarks=t})})}]).controller("JobPlanController",["$scope","$state","$stateParams","$window","JobsService",function(e,t,r,n,i){return e.nodeid=null,e.nodeUnfolded=!1,e.stateList=i.stateList(),e.changeNode=function(t){return t!==e.nodeid?(e.nodeid=t,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null,e.$broadcast("reload"),e.$broadcast("node:change",e.nodeid)):(e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null)},e.deactivateNode=function(){return e.nodeid=null,e.nodeUnfolded=!1,e.vertex=null,e.subtasks=null,e.accumulators=null,e.operatorCheckpointStats=null},e.toggleFold=function(){return e.nodeUnfolded=!e.nodeUnfolded}}]).controller("JobPlanSubtasksController",["$scope","JobsService",function(e,t){var r;return e.aggregate=!1,r=function(){return e.aggregate?t.getTaskManagers(e.nodeid).then(function(t){return e.taskmanagers=t}):t.getSubtasks(e.nodeid).then(function(t){return e.subtasks=t})},!e.nodeid||e.vertex&&e.vertex.st||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanAccumulatorsController",["$scope","JobsService",function(e,t){var r;return r=function(){return t.getAccumulators(e.nodeid).then(function(t){return e.accumulators=t.main,e.subtaskAccumulators=t.subtasks})},!e.nodeid||e.vertex&&e.vertex.accumulators||r(),e.$on("reload",function(t){if(e.nodeid)return r()})}]).controller("JobPlanCheckpointsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return e.checkpointDetails={},e.checkpointDetails.id=-1,n.getCheckpointConfig().then(function(t){return e.checkpointConfig=t}),i=function(){return n.getCheckpointStats().then(function(t){if(null!==t)return e.checkpointStats=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobPlanCheckpointDetailsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i,o;return e.subtaskDetails={},e.checkpointDetails.id=r.checkpointId,i=function(t){return n.getCheckpointDetails(t).then(function(t){return null!==t?e.checkpoint=t:e.unknown_checkpoint=!0})},o=function(t,r){return n.getCheckpointSubtaskDetails(t,r).then(function(t){if(null!==t)return e.subtaskDetails[r]=t})},i(r.checkpointId),e.nodeid&&o(r.checkpointId,e.nodeid),e.$on("reload",function(t){if(i(r.checkpointId),e.nodeid)return o(r.checkpointId,e.nodeid)}),e.$on("$destroy",function(){return e.checkpointDetails.id=-1})}]).controller("JobPlanBackPressureController",["$scope","JobsService",function(e,t){var r;return r=function(){if(e.now=Date.now(),e.nodeid)return t.getOperatorBackPressure(e.nodeid).then(function(t){return e.backPressureOperatorStats[e.nodeid]=t})},r(),e.$on("reload",function(e){return r()})}]).controller("JobTimelineVertexController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){var i;return i=function(){return n.getVertex(r.vertexId).then(function(t){return e.vertex=t})},i(),e.$on("reload",function(e){return i()})}]).controller("JobExceptionsController",["$scope","$state","$stateParams","JobsService",function(e,t,r,n){return n.loadExceptions().then(function(t){return e.exceptions=t})}]).controller("JobPropertiesController",["$scope","JobsService",function(e,t){return e.changeNode=function(r){return r!==e.nodeid?(e.nodeid=r,t.getNode(r).then(function(t){return e.node=t})):(e.nodeid=null,e.node=null)}}]).controller("JobPlanMetricsController",["$scope","JobsService","MetricsService",function(e,t,r){var n,i;if(e.dragging=!1,e.window=r.getWindow(),e.availableMetrics=null,e.$on("$destroy",function(){return r.unRegisterObserver()}),i=function(){return t.getVertex(e.nodeid).then(function(t){return e.vertex=t}),r.getAvailableMetrics(e.jobid,e.nodeid).then(function(t){return e.availableMetrics=t.sort(n),e.metrics=r.getMetricsSetup(e.jobid,e.nodeid).names,r.registerObserver(e.jobid,e.nodeid,function(t){return e.$broadcast("metrics:data:update",t.timestamp,t.values)})})},n=function(e,t){var r,n;return r=e.id.toLowerCase(),n=t.id.toLowerCase(),rn?1:0},e.dropped=function(t,n,o,a,s){return r.orderMetrics(e.jobid,e.nodeid,o,n),e.$broadcast("metrics:refresh",o),i(),!1},e.dragStart=function(){return e.dragging=!0},e.dragEnd=function(){return e.dragging=!1},e.addMetric=function(t){return r.addMetric(e.jobid,e.nodeid,t.id),i()},e.removeMetric=function(t){return r.removeMetric(e.jobid,e.nodeid,t),i()},e.setMetricSize=function(t,n){return r.setMetricSize(e.jobid,e.nodeid,t,n),i()},e.setMetricView=function(t,n){return r.setMetricView(e.jobid,e.nodeid,t,n),i()},e.getValues=function(t){return r.getValues(e.jobid,e.nodeid,t)},e.$on("node:change",function(t,r){if(!e.dragging)return i()}),e.nodeid)return i()}]),angular.module("flinkApp").directive("vertex",["$state",function(e){return{template:"",scope:{data:"="},link:function(e,t,r){var n,i,o;o=t.children()[0],i=t.width(),angular.element(o).attr("width",i),(n=function(e){var t,r,n;return d3.select(o).selectAll("*").remove(),n=[],angular.forEach(e.subtasks,function(e,t){var r;return r=[{label:"Scheduled",color:"#666",borderColor:"#555",starting_time:e.timestamps.SCHEDULED,ending_time:e.timestamps.DEPLOYING,type:"regular"},{label:"Deploying",color:"#aaa",borderColor:"#555",starting_time:e.timestamps.DEPLOYING,ending_time:e.timestamps.RUNNING,type:"regular"}],e.timestamps.FINISHED>0&&r.push({label:"Running",color:"#ddd",borderColor:"#555",starting_time:e.timestamps.RUNNING,ending_time:e.timestamps.FINISHED,type:"regular"}),n.push({label:"("+e.subtask+") "+e.host,times:r})}),t=d3.timeline().stack().tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("single").labelFormat(function(e){return e}).margin({left:100,right:0,top:0,bottom:0}).itemHeight(30).relativeTime(),r=d3.select(o).datum(n).call(t)})(e.data)}}}]).directive("timeline",["$state",function(e){return{template:"",scope:{vertices:"=",jobid:"="},link:function(t,r,n){var i,o,a,s;a=r.children()[0],o=r.width(),angular.element(a).attr("width",o),s=function(e){return e.replace(">",">")},i=function(r){var n,i,o;return d3.select(a).selectAll("*").remove(),o=[],angular.forEach(r,function(e){if(e["start-time"]>-1)return"scheduled"===e.type?o.push({times:[{label:s(e.name),color:"#cccccc",borderColor:"#555555",starting_time:e["start-time"],ending_time:e["end-time"],type:e.type}]}):o.push({times:[{label:s(e.name),color:"#d9f1f7",borderColor:"#62cdea",starting_time:e["start-time"],ending_time:e["end-time"],link:e.id,type:e.type}]})}),n=d3.timeline().stack().click(function(r,n,i){if(r.link)return e.go("single-job.timeline.vertex",{jobid:t.jobid,vertexId:r.link})}).tickFormat({format:d3.time.format("%L"),tickSize:1}).prefix("main").margin({left:0,right:0,top:0,bottom:0}).itemHeight(30).showBorderLine().showHourTimeline(),i=d3.select(a).datum(o).call(n)},t.$watch(n.vertices,function(e){if(e)return i(e)})}}}]).directive("split",function(){return{compile:function(e,t){return Split(e.children(),{sizes:[50,50],direction:"vertical"})}}}).directive("jobPlan",["$timeout",function(e){return{template:"
",scope:{plan:"=",watermarks:"=",setNode:"&"},link:function(e,t,r){var n,i,o,a,s,l,u,c,d,f,p,m,g,h,b,v,k,j,S,w,C,$,M,y,J;p=null,C=d3.behavior.zoom(),J=[],h=r.jobid,S=t.children()[0],j=t.children().children()[0],w=t.children()[1],l=d3.select(S),u=d3.select(j),c=d3.select(w),n=t.width(),angular.element(t.children()[0]).width(n),v=0,b=0,e.zoomIn=function(){var e,t,r;if(C.scale()<2.99)return e=C.translate(),t=e[0]*(C.scale()+.1/C.scale()),r=e[1]*(C.scale()+.1/C.scale()),C.scale(C.scale()+.1),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},e.zoomOut=function(){var e,t,r;if(C.scale()>.31)return C.scale(C.scale()-.1),e=C.translate(),t=e[0]*(C.scale()-.1/C.scale()),r=e[1]*(C.scale()-.1/C.scale()),C.translate([t,r]),u.attr("transform","translate("+t+","+r+") scale("+C.scale()+")"),v=C.scale(),b=C.translate()},o=function(e){var t;return t="",null==e.ship_strategy&&null==e.local_strategy||(t+="
",null!=e.ship_strategy&&(t+=e.ship_strategy),void 0!==e.temp_mode&&(t+=" ("+e.temp_mode+")"),void 0!==e.local_strategy&&(t+=",
"+e.local_strategy),t+="
"),t},g=function(e){return"partialSolution"===e||"nextPartialSolution"===e||"workset"===e||"nextWorkset"===e||"solutionSet"===e||"solutionDelta"===e},m=function(e,t){return"mirror"===t?"node-mirror":g(t)?"node-iteration":"node-normal"},a=function(e,t,r,n){var i,o;return i="
",i+="mirror"===t?"

Mirror of "+e.operator+"

":"

"+e.operator+"

",""===e.description?i+="":(o=e.description,o=y(o),i+="

"+o+"

"),null!=e.step_function?i+=f(e.id,r,n):(g(t)&&(i+="
"+t+" Node
"),""!==e.parallelism&&(i+="
Parallelism: "+e.parallelism+"
"),void 0!==e.lowWatermark&&(i+="
Low Watermark: "+e.lowWatermark+"
"),void 0!==e.operator&&e.operator_strategy&&(i+="
Operation: "+y(e.operator_strategy)+"
")),i+="
"},f=function(e,t,r){var n,i;return i="svg-"+e,n=""},y=function(e){var t;for("<"===e.charAt(0)&&(e=e.replace("<","<"),e=e.replace(">",">")),t="";e.length>30;)t=t+e.substring(0,30)+"
",e=e.substring(30,e.length);return t+=e},s=function(e,t,r,n,i,o){return null==n&&(n=!1),r.id===t.partial_solution?e.setNode(r.id,{label:a(r,"partialSolution",i,o),labelType:"html","class":m(r,"partialSolution")}):r.id===t.next_partial_solution?e.setNode(r.id,{label:a(r,"nextPartialSolution",i,o),labelType:"html","class":m(r,"nextPartialSolution")}):r.id===t.workset?e.setNode(r.id,{label:a(r,"workset",i,o),labelType:"html","class":m(r,"workset")}):r.id===t.next_workset?e.setNode(r.id,{label:a(r,"nextWorkset",i,o),labelType:"html","class":m(r,"nextWorkset")}):r.id===t.solution_set?e.setNode(r.id,{label:a(r,"solutionSet",i,o),labelType:"html","class":m(r,"solutionSet")}):r.id===t.solution_delta?e.setNode(r.id,{label:a(r,"solutionDelta",i,o),labelType:"html","class":m(r,"solutionDelta")}):e.setNode(r.id,{label:a(r,"",i,o),labelType:"html","class":m(r,"")})},i=function(e,t,r,n,i){return e.setEdge(i.id,r.id,{label:o(i),labelType:"html",arrowhead:"normal"})},k=function(e,t){var r,n,o,a,l,u,d,f,p,m,g,h,b,v;for(n=[],null!=t.nodes?v=t.nodes:(v=t.step_function,o=!0),a=0,u=v.length;a-1))return e["end-time"]=e["start-time"]+e.duration})},this.processVertices=function(e){return angular.forEach(e.vertices,function(e,t){return e.type="regular"}),e.vertices.unshift({name:"Scheduled","start-time":e.timestamps.CREATED,"end-time":e.timestamps.CREATED+1,type:"scheduled"})},this.listJobs=function(){var r;return r=i.defer(),e.get(t.jobServer+"jobs/overview").success(function(e){return function(t,n,i,o){return c.finished=[],c.running=[],_(t.jobs).groupBy(function(e){switch(e.state.toLowerCase()){case"finished":return"finished";case"failed":return"finished";case"canceled":return"finished";default:return"running"}}).forEach(function(t,r){switch(r){case"finished":return c.finished=e.setEndTimes(t);case"running":return c.running=e.setEndTimes(t)}}).value(),r.resolve(c),d()}}(this)),r.promise},this.getJobs=function(e){return c[e]},this.getAllJobs=function(){return c},this.loadJob=function(r){return a=null,l.job=i.defer(),e.get(t.jobServer+"jobs/"+r).success(function(n){return function(i,o,s,u){return n.setEndTimes(i.vertices),n.processVertices(i),e.get(t.jobServer+"jobs/"+r+"/config").success(function(e){return i=angular.extend(i,e),a=i,l.job.resolve(a)})}}(this)),l.job.promise},this.getNode=function(e){var t,r;return r=function(e,t){var n,i,o,a;for(n=0,i=t.length;n
{{metric.id}}
{{value | humanizeChartNumeric:metric}}
',replace:!0,scope:{metric:"=",window:"=",removeMetric:"&",setMetricSize:"=",setMetricView:"=",getValues:"&"},link:function(e,t,r){return e.btnClasses=["btn","btn-default","btn-xs"],e.value=null,e.data=[{values:e.getValues()}],e.options={x:function(e,t){return e.x},y:function(e,t){return e.y},xTickFormat:function(e){return d3.time.format("%H:%M:%S")(new Date(e))},yTickFormat:function(e){var t,r,n,i;for(r=!1,n=0,i=1,t=Math.abs(e);!r&&n<50;)Math.pow(10,n)<=t&&t6?e/Math.pow(10,n)+"E"+n:""+e}},e.showChart=function(){return d3.select(t.find("svg")[0]).datum(e.data).transition().duration(250).call(e.chart)},e.chart=nv.models.lineChart().options(e.options).showLegend(!1).margin({top:15,left:60,bottom:30,right:30}),e.chart.yAxis.showMaxMin(!1),e.chart.tooltip.hideDelay(0),e.chart.tooltip.contentGenerator(function(e){return"

"+d3.time.format("%H:%M:%S")(new Date(e.point.x))+" | "+e.point.y+"

"}),nv.utils.windowResize(e.chart.update),e.setSize=function(t){return e.setMetricSize(e.metric,t)},e.setView=function(t){if(e.setMetricView(e.metric,t),"chart"===t)return e.showChart()},"chart"===e.metric.view&&e.showChart(),e.$on("metrics:data:update",function(t,r,n){return e.value=parseFloat(n[e.metric.id]),e.data[0].values.push({x:r,y:e.value}),e.data[0].values.length>e.window&&e.data[0].values.shift(),"chart"===e.metric.view&&e.showChart(),"chart"===e.metric.view&&e.chart.clearHighlights(),e.chart.tooltip.hidden(!0)}),t.find(".metric-title").qtip({content:{text:e.metric.id},position:{my:"bottom left",at:"top left"},style:{classes:"qtip-light qtip-timeline-bar"}})}}}),angular.module("flinkApp").service("MetricsService",["$http","$q","flinkConfig","$interval",function(e,t,r,n){return this.metrics={},this.values={},this.watched={},this.observer={jobid:null,nodeid:null,callback:null},this.refresh=n(function(e){return function(){return angular.forEach(e.metrics,function(t,r){return angular.forEach(t,function(t,n){var i;if(i=[],angular.forEach(t,function(e,t){return i.push(e.id)}),i.length>0)return e.getMetrics(r,n,i).then(function(t){if(r===e.observer.jobid&&n===e.observer.nodeid&&e.observer.callback)return e.observer.callback(t)})})})}}(this),r["refresh-interval"]),this.registerObserver=function(e,t,r){return this.observer.jobid=e,this.observer.nodeid=t,this.observer.callback=r},this.unRegisterObserver=function(){return this.observer={jobid:null,nodeid:null,callback:null}},this.setupMetrics=function(e,t){return this.setupLS(),this.watched[e]=[],angular.forEach(t,function(t){return function(r,n){if(r.id)return t.watched[e].push(r.id)}}(this))},this.getWindow=function(){return 100},this.setupLS=function(){return null==sessionStorage.flinkMetrics&&this.saveSetup(),this.metrics=JSON.parse(sessionStorage.flinkMetrics)},this.saveSetup=function(){return sessionStorage.flinkMetrics=JSON.stringify(this.metrics)},this.saveValue=function(e,t,r){if(null==this.values[e]&&(this.values[e]={}),null==this.values[e][t]&&(this.values[e][t]=[]),this.values[e][t].push(r),this.values[e][t].length>this.getWindow())return this.values[e][t].shift()},this.getValues=function(e,t,r){var n;return null==this.values[e]?[]:null==this.values[e][t]?[]:(n=[],angular.forEach(this.values[e][t],function(e){return function(e,t){if(null!=e.values[r])return n.push({x:e.timestamp,y:e.values[r]})}}(this)),n)},this.setupLSFor=function(e,t){if(null==this.metrics[e]&&(this.metrics[e]={}),null==this.metrics[e][t])return this.metrics[e][t]=[]},this.addMetric=function(e,t,r){return this.setupLSFor(e,t),this.metrics[e][t].push({id:r,size:"small",view:"chart"}),this.saveSetup()},this.removeMetric=function(e){return function(t,r,n){var i;if(null!=e.metrics[t][r])return i=e.metrics[t][r].indexOf(n),i===-1&&(i=_.findIndex(e.metrics[t][r],{id:n})),i!==-1&&e.metrics[t][r].splice(i,1),e.saveSetup()}}(this),this.setMetricSize=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:i,view:n.view}),e.saveSetup()}}(this),this.setMetricView=function(e){return function(t,r,n,i){var o;if(null!=e.metrics[t][r])return o=e.metrics[t][r].indexOf(n.id),o===-1&&(o=_.findIndex(e.metrics[t][r],{id:n.id})),o!==-1&&(e.metrics[t][r][o]={id:n.id,size:n.size,view:i}),e.saveSetup()}}(this),this.orderMetrics=function(e,t,r,n){return this.setupLSFor(e,t),angular.forEach(this.metrics[e][t],function(i){return function(o,a){if(o.id===r.id&&(i.metrics[e][t].splice(a,1),a { + private final TerminationModeQueryParameter.TerminationMode defaultTerminationMode; + public JobTerminationHandler( CompletableFuture localRestAddress, GatewayRetriever leaderRetriever, Time timeout, Map headers, - MessageHeaders messageHeaders) { + MessageHeaders messageHeaders, + TerminationModeQueryParameter.TerminationMode defaultTerminationMode) { super(localRestAddress, leaderRetriever, timeout, headers, messageHeaders); + + this.defaultTerminationMode = Preconditions.checkNotNull(defaultTerminationMode); } @Override @@ -65,7 +71,7 @@ public CompletableFuture handleRequest(HandlerRequestFor more information @see YARN-2031. + * + * @deprecated This should be removed once we can send arbitrary REST calls via the Yarn proxy. + */ +@Deprecated +public class YarnCancelJobTerminationHeaders implements RestHandlerSpecification { + + private static final YarnCancelJobTerminationHeaders INSTANCE = new YarnCancelJobTerminationHeaders(); + + private static final String URL = String.format("/jobs/:%s/yarn-cancel", JobIDPathParameter.KEY); + + private YarnCancelJobTerminationHeaders() {} + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static YarnCancelJobTerminationHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeaders.java new file mode 100644 index 0000000000000..bbd36c0795311 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeaders.java @@ -0,0 +1,55 @@ +/* + * 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.rest.messages; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; +import org.apache.flink.runtime.rest.handler.job.JobTerminationHandler; + +/** + * {@link RestHandlerSpecification} for the {@link JobTerminationHandler} which is registered for + * compatibility with the Yarn proxy as a GET call. + * + *

For more information @see YARN-2031. + * + * @deprecated This should be removed once we can send arbitrary REST calls via the Yarn proxy. + */ +@Deprecated +public class YarnStopJobTerminationHeaders implements RestHandlerSpecification { + + private static final YarnStopJobTerminationHeaders INSTANCE = new YarnStopJobTerminationHeaders(); + + private static final String URL = String.format("/jobs/:%s/yarn-stop", JobIDPathParameter.KEY); + + private YarnStopJobTerminationHeaders() {} + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static YarnStopJobTerminationHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 5aa3271973c79..19efcbb148346 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -85,6 +85,9 @@ import org.apache.flink.runtime.rest.messages.JobVertexTaskManagersHeaders; import org.apache.flink.runtime.rest.messages.JobsOverviewHeaders; import org.apache.flink.runtime.rest.messages.SubtasksTimesHeaders; +import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter; +import org.apache.flink.runtime.rest.messages.YarnCancelJobTerminationHeaders; +import org.apache.flink.runtime.rest.messages.YarnStopJobTerminationHeaders; import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigHeaders; import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointStatisticDetailsHeaders; import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointingStatisticsHeaders; @@ -451,12 +454,21 @@ protected List> initiali responseHeaders, JobVertexBackPressureHeaders.getInstance()); - final JobTerminationHandler jobTerminationHandler = new JobTerminationHandler( + final JobTerminationHandler jobCancelTerminationHandler = new JobTerminationHandler( restAddressFuture, leaderRetriever, timeout, responseHeaders, - JobTerminationHeaders.getInstance()); + JobTerminationHeaders.getInstance(), + TerminationModeQueryParameter.TerminationMode.CANCEL); + + final JobTerminationHandler jobStopTerminationHandler = new JobTerminationHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders, + JobTerminationHeaders.getInstance(), + TerminationModeQueryParameter.TerminationMode.STOP); final File tmpDir = restConfiguration.getTmpDir(); @@ -504,7 +516,11 @@ protected List> initiali handlers.add(Tuple2.of(SubtaskCurrentAttemptDetailsHeaders.getInstance(), subtaskCurrentAttemptDetailsHandler)); handlers.add(Tuple2.of(JobVertexTaskManagersHeaders.getInstance(), jobVertexTaskManagersHandler)); handlers.add(Tuple2.of(JobVertexBackPressureHeaders.getInstance(), jobVertexBackPressureHandler)); - handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobTerminationHandler)); + handlers.add(Tuple2.of(JobTerminationHeaders.getInstance(), jobCancelTerminationHandler)); + + // TODO: Remove once the Yarn proxy can forward all REST verbs + handlers.add(Tuple2.of(YarnCancelJobTerminationHeaders.getInstance(), jobCancelTerminationHandler)); + handlers.add(Tuple2.of(YarnStopJobTerminationHeaders.getInstance(), jobStopTerminationHandler)); optWebContent.ifPresent( webContent -> handlers.add(Tuple2.of(WebContentHandlerSpecification.getInstance(), webContent))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnCancelJobTerminationHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnCancelJobTerminationHeadersTest.java new file mode 100644 index 0000000000000..f1a016bf31202 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnCancelJobTerminationHeadersTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.rest.messages; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for the {@link YarnCancelJobTerminationHeaders}. + */ +public class YarnCancelJobTerminationHeadersTest extends TestLogger { + + // instance under test + private static final YarnCancelJobTerminationHeaders instance = YarnCancelJobTerminationHeaders.getInstance(); + + @Test + public void testMethod() { + assertThat(instance.getHttpMethod(), is(HttpMethodWrapper.GET)); + } + + @Test + public void testURL() { + assertThat(instance.getTargetRestEndpointURL(), endsWith("yarn-cancel")); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeadersTest.java new file mode 100644 index 0000000000000..a5d814f18d1de --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/YarnStopJobTerminationHeadersTest.java @@ -0,0 +1,48 @@ +/* + * 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.rest.messages; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link YarnStopJobTerminationHeaders}. + */ +public class YarnStopJobTerminationHeadersTest extends TestLogger { + + // instance under test + private static final YarnStopJobTerminationHeaders instance = YarnStopJobTerminationHeaders.getInstance(); + + @Test + public void testMethod() { + assertThat(instance.getHttpMethod(), is(HttpMethodWrapper.GET)); + } + + @Test + public void testURL() { + assertThat(instance.getTargetRestEndpointURL(), endsWith("yarn-stop")); + } + +} From 3eec3158d08caf6d7dcf617c5d74842e1e5b083d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 14 Feb 2018 15:44:06 +0100 Subject: [PATCH 6/8] [FLINK-8608] [flip6] Implement MiniDispatcher for job mode The MiniDispatcher is responsible for submitting the single job with which a job mode cluster is started. Once the job has completed and if the cluster has been started in detached mode, the MiniDispatcher will terminate. In order to reduce code duplication, the MiniDispatcher is a sub class of the Dispatcher which is started with a single job submitted job graph store. This closes #5431. --- .../client/deployment/ClusterDescriptor.java | 5 +- .../Flip6StandaloneClusterDescriptor.java | 5 +- .../StandaloneClusterDescriptor.java | 5 +- .../cli/util/DummyClusterDescriptor.java | 5 +- .../entrypoint/MesosJobClusterEntrypoint.java | 16 +- .../MesosSessionClusterEntrypoint.java | 16 +- .../flink/runtime/dispatcher/Dispatcher.java | 112 +++-- .../MemoryArchivedExecutionGraphStore.java | 0 .../runtime/dispatcher/MiniDispatcher.java | 120 ++++++ .../SingleJobSubmittedJobGraphStore.java | 77 ++++ .../dispatcher/StandaloneDispatcher.java | 36 +- .../runtime/entrypoint/ClusterEntrypoint.java | 383 ++++++++++++++++-- .../entrypoint/JobClusterEntrypoint.java | 301 ++------------ .../entrypoint/SessionClusterEntrypoint.java | 202 +-------- ...t.java => MiniDispatcherRestEndpoint.java} | 10 +- .../handler/job/BlobServerPortHandler.java | 2 +- .../rest/handler/job/JobSubmitHandler.java | 2 +- .../legacy/files/StaticFileServerHandler.java | 2 +- .../TaskExecutorRegistrationSuccess.java | 2 +- .../taskexecutor/slot/TaskSlotTable.java | 2 +- .../webmonitor/WebMonitorEndpoint.java | 6 +- .../runtime/webmonitor/WebMonitorUtils.java | 2 +- .../runtime/dispatcher/DispatcherTest.java | 54 ++- .../dispatcher/MiniDispatcherTest.java | 211 ++++++++++ .../yarn/TestingYarnClusterDescriptor.java | 5 +- .../org/apache/flink/yarn/YARNITCase.java | 5 +- .../yarn/AbstractYarnClusterDescriptor.java | 51 +-- .../yarn/Flip6YarnClusterDescriptor.java | 24 ++ .../flink/yarn/YarnClusterDescriptor.java | 5 +- .../flink/yarn/AbstractYarnClusterTest.java | 8 + 30 files changed, 1023 insertions(+), 651 deletions(-) rename flink-runtime/src/{test => main}/java/org/apache/flink/runtime/dispatcher/MemoryArchivedExecutionGraphStore.java (100%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java rename flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/{JobMasterRestEndpoint.java => MiniDispatcherRestEndpoint.java} (86%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java index aadb5feb3ced3..f9f5d4b5e1011 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/ClusterDescriptor.java @@ -56,12 +56,15 @@ public interface ClusterDescriptor extends AutoCloseable { * * @param clusterSpecification Initial cluster specification with which the Flink cluster is launched * @param jobGraph JobGraph with which the job cluster is started + * @param detached true if the cluster should be stopped after the job completion without serving the result, + * otherwise false * @return Cluster client to talk to the Flink cluster * @throws ClusterDeploymentException if the cluster could not be deployed */ ClusterClient deployJobCluster( final ClusterSpecification clusterSpecification, - final JobGraph jobGraph) throws ClusterDeploymentException; + final JobGraph jobGraph, + final boolean detached) throws ClusterDeploymentException; /** * Terminates the cluster with the given cluster id. diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java index 8096da8701e5f..0a3286a3cc9b3 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java @@ -58,7 +58,10 @@ public RestClusterClient deploySessionCluster(ClusterSpecif } @Override - public RestClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { + public RestClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { throw new UnsupportedOperationException("Can't deploy a standalone FLIP-6 per-job cluster."); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java index 62908fe28f9b6..83bb13646f203 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java @@ -57,7 +57,10 @@ public StandaloneClusterClient deploySessionCluster(ClusterSpecification cluster } @Override - public StandaloneClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { + public StandaloneClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { throw new UnsupportedOperationException("Can't deploy a standalone per-job cluster."); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java index 0ceb95edad24f..df2f3f78591a1 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/util/DummyClusterDescriptor.java @@ -52,7 +52,10 @@ public ClusterClient deploySessionCluster(ClusterSpecification clusterSpecifi } @Override - public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { + public ClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { return clusterClient; } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 21864dcf232ed..b0054bb52e598 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -25,7 +25,6 @@ import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; -import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -113,17 +112,6 @@ protected void initializeServices(Configuration config) throws Exception { taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); } - @Override - protected void startClusterComponents( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { - super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); - } - @Override protected ResourceManager createResourceManager( Configuration configuration, @@ -179,11 +167,11 @@ protected JobGraph retrieveJobGraph(Configuration configuration) throws FlinkExc } @Override - protected void stopClusterComponents(boolean cleanupHaData) throws Exception { + protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { Throwable exception = null; try { - super.stopClusterComponents(cleanupHaData); + super.stopClusterServices(cleanupHaData); } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 68cf7c6b9cdaa..306d5b2b2a5aa 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -25,7 +25,6 @@ import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.util.MesosConfiguration; -import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -103,17 +102,6 @@ protected void initializeServices(Configuration config) throws Exception { taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); } - @Override - protected void startClusterComponents( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { - super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); - } - @Override protected ResourceManager createResourceManager( Configuration configuration, @@ -153,11 +141,11 @@ protected ResourceManager createResourceManager( } @Override - protected void stopClusterComponents(boolean cleanupHaData) throws Exception { + protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { Throwable exception = null; try { - super.stopClusterComponents(cleanupHaData); + super.stopClusterServices(cleanupHaData); } catch (Throwable t) { exception = t; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index c83dce0bc6db8..92ccc9b9fd9ac 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -102,19 +102,23 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final ArchivedExecutionGraphStore archivedExecutionGraphStore; + private final JobManagerRunnerFactory jobManagerRunnerFactory; + @Nullable protected final String restAddress; - protected Dispatcher( + public Dispatcher( RpcService rpcService, String endpointId, Configuration configuration, HighAvailabilityServices highAvailabilityServices, + SubmittedJobGraphStore submittedJobGraphStore, ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, ArchivedExecutionGraphStore archivedExecutionGraphStore, + JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { super(rpcService, endpointId); @@ -126,12 +130,12 @@ protected Dispatcher( this.blobServer = Preconditions.checkNotNull(blobServer); this.metricRegistry = Preconditions.checkNotNull(metricRegistry); this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler); + this.submittedJobGraphStore = Preconditions.checkNotNull(submittedJobGraphStore); this.jobManagerSharedServices = JobManagerSharedServices.fromConfiguration( configuration, this.blobServer); - this.submittedJobGraphStore = highAvailabilityServices.getSubmittedJobGraphStore(); this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry(); jobManagerRunners = new HashMap<>(16); @@ -141,6 +145,8 @@ protected Dispatcher( this.restAddress = restAddress; this.archivedExecutionGraphStore = Preconditions.checkNotNull(archivedExecutionGraphStore); + + this.jobManagerRunnerFactory = Preconditions.checkNotNull(jobManagerRunnerFactory); } //------------------------------------------------------ @@ -226,7 +232,7 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) final JobManagerRunner jobManagerRunner; try { - jobManagerRunner = createJobManagerRunner( + jobManagerRunner = jobManagerRunnerFactory.createJobManagerRunner( ResourceID.generate(), jobGraph, configuration, @@ -237,7 +243,8 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) jobManagerSharedServices, metricRegistry, new DispatcherOnCompleteActions(jobGraph.getJobID()), - fatalErrorHandler); + fatalErrorHandler, + restAddress); jobManagerRunner.start(); } catch (Exception e) { @@ -507,16 +514,18 @@ void recoverJobs() { }); } - private void onFatalError(Throwable throwable) { + protected void onFatalError(Throwable throwable) { log.error("Fatal error occurred in dispatcher {}.", getAddress(), throwable); + fatalErrorHandler.onFatalError(throwable); } - private void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedExecutionGraph) { + protected void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedExecutionGraph) { Preconditions.checkArgument( archivedExecutionGraph.getState().isGloballyTerminalState(), - "Job " + archivedExecutionGraph.getJobID() + " is in state " + - archivedExecutionGraph.getState() + " which is not globally terminal."); + "Job %s is in state %s which is not globally terminal.", + archivedExecutionGraph.getJobID(), + archivedExecutionGraph.getState()); try { archivedExecutionGraphStore.put(archivedExecutionGraph); @@ -537,18 +546,13 @@ private void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedExec } } - protected abstract JobManagerRunner createJobManagerRunner( - ResourceID resourceId, - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - BlobServer blobServer, - JobManagerSharedServices jobManagerSharedServices, - MetricRegistry metricRegistry, - OnCompletionActions onCompleteActions, - FatalErrorHandler fatalErrorHandler) throws Exception; + protected void jobFinishedByOther(JobID jobId) { + try { + removeJob(jobId, false); + } catch (Exception e) { + log.warn("Could not properly remove job {} from the dispatcher.", jobId, e); + } + } //------------------------------------------------------ // Leader contender @@ -671,13 +675,67 @@ public void jobFinishedByOther() { log.info("Job {} was finished by other JobManager.", jobId); runAsync( - () -> { - try { - removeJob(jobId, false); - } catch (Exception e) { - log.warn("Could not properly remove job {} from the dispatcher.", jobId, e); - } - }); + () -> Dispatcher.this.jobFinishedByOther(jobId)); + } + } + + //------------------------------------------------------ + // Factories + //------------------------------------------------------ + + /** + * Factory for a {@link JobManagerRunner}. + */ + @FunctionalInterface + public interface JobManagerRunnerFactory { + JobManagerRunner createJobManagerRunner( + ResourceID resourceId, + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + JobManagerSharedServices jobManagerServices, + MetricRegistry metricRegistry, + OnCompletionActions onCompleteActions, + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress) throws Exception; + } + + /** + * Singleton default factory for {@link JobManagerRunner}. + */ + public enum DefaultJobManagerRunnerFactory implements JobManagerRunnerFactory { + INSTANCE; + + @Override + public JobManagerRunner createJobManagerRunner( + ResourceID resourceId, + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + JobManagerSharedServices jobManagerServices, + MetricRegistry metricRegistry, + OnCompletionActions onCompleteActions, + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress) throws Exception { + return new JobManagerRunner( + resourceId, + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + blobServer, + jobManagerServices, + metricRegistry, + onCompleteActions, + fatalErrorHandler, + restAddress); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryArchivedExecutionGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MemoryArchivedExecutionGraphStore.java similarity index 100% rename from flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryArchivedExecutionGraphStore.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MemoryArchivedExecutionGraphStore.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java new file mode 100644 index 0000000000000..26d357e7ba52b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -0,0 +1,120 @@ +/* + * 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.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.FlinkException; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; + +/** + * Mini Dispatcher which is instantiated as the dispatcher component by the {@link JobClusterEntrypoint}. + * + *

The mini dispatcher is initialized with a single {@link JobGraph} which it runs. + * + *

Depending on the {@link ClusterEntrypoint.ExecutionMode}, the mini dispatcher will directly + * terminate after job completion if its execution mode is {@link ClusterEntrypoint.ExecutionMode#DETACHED}. + */ +public class MiniDispatcher extends Dispatcher { + + private final JobClusterEntrypoint.ExecutionMode executionMode; + + public MiniDispatcher( + RpcService rpcService, + String endpointId, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + ResourceManagerGateway resourceManagerGateway, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + MetricRegistry metricRegistry, + ArchivedExecutionGraphStore archivedExecutionGraphStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress, + JobGraph jobGraph, + JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { + super( + rpcService, + endpointId, + configuration, + highAvailabilityServices, + new SingleJobSubmittedJobGraphStore(jobGraph), + resourceManagerGateway, + blobServer, + heartbeatServices, + metricRegistry, + archivedExecutionGraphStore, + jobManagerRunnerFactory, + fatalErrorHandler, + restAddress); + + this.executionMode = executionMode; + } + + @Override + public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) { + final CompletableFuture acknowledgeCompletableFuture = super.submitJob(jobGraph, timeout); + + acknowledgeCompletableFuture.whenComplete( + (Acknowledge ignored, Throwable throwable) -> { + if (throwable != null) { + onFatalError(new FlinkException( + "Failed to submit job " + jobGraph.getJobID() + " in job mode.", + throwable)); + } + }); + + return acknowledgeCompletableFuture; + } + + @Override + protected void jobReachedGloballyTerminalState(ArchivedExecutionGraph archivedExecutionGraph) { + super.jobReachedGloballyTerminalState(archivedExecutionGraph); + + if (executionMode == ClusterEntrypoint.ExecutionMode.DETACHED) { + // shut down since we don't have to wait for the execution result retrieval + shutDown(); + } + } + + @Override + protected void jobFinishedByOther(JobID jobId) { + super.jobFinishedByOther(jobId); + + // shut down since we have done our job + shutDown(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java new file mode 100644 index 0000000000000..26d3abc2f1fd0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobSubmittedJobGraphStore.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraph; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@link SubmittedJobGraphStore} implementation for a single job. + */ +public class SingleJobSubmittedJobGraphStore implements SubmittedJobGraphStore { + + private final JobGraph jobGraph; + + public SingleJobSubmittedJobGraphStore(JobGraph jobGraph) { + this.jobGraph = Preconditions.checkNotNull(jobGraph); + } + + @Override + public void start(SubmittedJobGraphListener jobGraphListener) throws Exception { + // noop + } + + @Override + public void stop() throws Exception { + // noop + } + + @Override + public SubmittedJobGraph recoverJobGraph(JobID jobId) throws Exception { + if (jobGraph.getJobID().equals(jobId)) { + return new SubmittedJobGraph(jobGraph, null); + } else { + throw new FlinkException("Could not recover job graph " + jobId + '.'); + } + } + + @Override + public void putJobGraph(SubmittedJobGraph jobGraph) throws Exception { + if (!jobGraph.getJobId().equals(jobGraph.getJobId())) { + throw new FlinkException("Cannot put additional jobs into this submitted job graph store."); + } + } + + @Override + public void removeJobGraph(JobID jobId) throws Exception { + // ignore + } + + @Override + public Collection getJobIds() throws Exception { + return Collections.singleton(jobGraph.getJobID()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index 6592e98c38a49..a7d21f3faaa08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -20,13 +20,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -51,6 +47,7 @@ public StandaloneDispatcher( HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, ArchivedExecutionGraphStore archivedExecutionGraphStore, + JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { super( @@ -58,40 +55,13 @@ public StandaloneDispatcher( endpointId, configuration, highAvailabilityServices, + highAvailabilityServices.getSubmittedJobGraphStore(), resourceManagerGateway, blobServer, heartbeatServices, metricRegistry, archivedExecutionGraphStore, - fatalErrorHandler, - restAddress); - } - - @Override - protected JobManagerRunner createJobManagerRunner( - ResourceID resourceId, - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - BlobServer blobServer, - JobManagerSharedServices jobManagerSharedServices, - MetricRegistry metricRegistry, - OnCompletionActions onCompleteActions, - FatalErrorHandler fatalErrorHandler) throws Exception { - // create the standard job manager runner - return new JobManagerRunner( - resourceId, - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - blobServer, - jobManagerSharedServices, - metricRegistry, - onCompleteActions, + jobManagerRunnerFactory, fatalErrorHandler, restAddress); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index dfb1b4ce9e2b1..584fcaee8d766 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -20,25 +20,48 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.TransientBlobCache; +import org.apache.flink.runtime.blob.TransientBlobService; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.dispatcher.MiniDispatcher; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; +import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; +import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; +import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; +import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -47,12 +70,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; import scala.concurrent.duration.FiniteDuration; @@ -63,6 +89,10 @@ */ public abstract class ClusterEntrypoint implements FatalErrorHandler { + public static final ConfigOption EXECUTION_MODE = ConfigOptions + .key("internal.cluster.execution-mode") + .defaultValue(ExecutionMode.NORMAL.toString()); + protected static final Logger LOG = LoggerFactory.getLogger(ClusterEntrypoint.class); protected static final int SUCCESS_RETURN_CODE = 0; @@ -76,20 +106,43 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { private final CompletableFuture terminationFuture; + private final AtomicBoolean isTerminating = new AtomicBoolean(false); + @GuardedBy("lock") - private MetricRegistryImpl metricRegistry = null; + private MetricRegistryImpl metricRegistry; @GuardedBy("lock") - private HighAvailabilityServices haServices = null; + private HighAvailabilityServices haServices; @GuardedBy("lock") - private BlobServer blobServer = null; + private BlobServer blobServer; @GuardedBy("lock") - private HeartbeatServices heartbeatServices = null; + private HeartbeatServices heartbeatServices; @GuardedBy("lock") - private RpcService commonRpcService = null; + private RpcService commonRpcService; + + @GuardedBy("lock") + private ResourceManager resourceManager; + + @GuardedBy("lock") + private Dispatcher dispatcher; + + @GuardedBy("lock") + private LeaderRetrievalService dispatcherLeaderRetrievalService; + + @GuardedBy("lock") + private LeaderRetrievalService resourceManagerRetrievalService; + + @GuardedBy("lock") + private WebMonitorEndpoint webMonitorEndpoint; + + @GuardedBy("lock") + private ArchivedExecutionGraphStore archivedExecutionGraphStore; + + @GuardedBy("lock") + private TransientBlobCache transientBlobCache; protected ClusterEntrypoint(Configuration configuration) { this.configuration = Preconditions.checkNotNull(configuration); @@ -108,24 +161,18 @@ protected void startCluster() { SecurityContext securityContext = installSecurityContext(configuration); - securityContext.runSecured(new Callable() { - @Override - public Void call() throws Exception { - runCluster(configuration); + securityContext.runSecured((Callable) () -> { + runCluster(configuration); - return null; - } + return null; }); } catch (Throwable t) { LOG.error("Cluster initialization failed.", t); - try { - shutDown(false); - } catch (Throwable st) { - LOG.error("Could not properly shut down cluster entrypoint.", st); - } - - System.exit(STARTUP_FAILURE_RETURN_CODE); + shutDownAndTerminate( + STARTUP_FAILURE_RETURN_CODE, + ApplicationStatus.FAILED, + false); } } @@ -163,33 +210,134 @@ protected void runCluster(Configuration configuration) throws Exception { blobServer, heartbeatServices, metricRegistry); + + // TODO: Make shutDownAndTerminate non blocking to not use the global executor + dispatcher.getTerminationFuture().whenCompleteAsync( + (Boolean success, Throwable throwable) -> { + if (throwable != null) { + LOG.info("Could not properly terminate the Dispatcher.", throwable); + } + + shutDownAndTerminate( + SUCCESS_RETURN_CODE, + ApplicationStatus.SUCCEEDED, + true); + }); } } protected void initializeServices(Configuration configuration) throws Exception { - assert(Thread.holdsLock(lock)); LOG.info("Initializing cluster services."); - final String bindAddress = configuration.getString(JobManagerOptions.ADDRESS); - final String portRange = getRPCPortRange(configuration); + synchronized (lock) { + final String bindAddress = configuration.getString(JobManagerOptions.ADDRESS); + final String portRange = getRPCPortRange(configuration); - commonRpcService = createRpcService(configuration, bindAddress, portRange); + commonRpcService = createRpcService(configuration, bindAddress, portRange); - // update the configuration used to create the high availability services - configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); - configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort()); + // update the configuration used to create the high availability services + configuration.setString(JobManagerOptions.ADDRESS, commonRpcService.getAddress()); + configuration.setInteger(JobManagerOptions.PORT, commonRpcService.getPort()); - haServices = createHaServices(configuration, commonRpcService.getExecutor()); - blobServer = new BlobServer(configuration, haServices.createBlobStore()); - blobServer.start(); - heartbeatServices = createHeartbeatServices(configuration); - metricRegistry = createMetricRegistry(configuration); + haServices = createHaServices(configuration, commonRpcService.getExecutor()); + blobServer = new BlobServer(configuration, haServices.createBlobStore()); + blobServer.start(); + heartbeatServices = createHeartbeatServices(configuration); + metricRegistry = createMetricRegistry(configuration); - // TODO: This is a temporary hack until we have ported the MetricQueryService to the new RpcEndpoint - // start the MetricQueryService - final ActorSystem actorSystem = ((AkkaRpcService) commonRpcService).getActorSystem(); - metricRegistry.startQueryService(actorSystem, null); + // TODO: This is a temporary hack until we have ported the MetricQueryService to the new RpcEndpoint + // start the MetricQueryService + final ActorSystem actorSystem = ((AkkaRpcService) commonRpcService).getActorSystem(); + metricRegistry.startQueryService(actorSystem, null); + } + } + + protected void startClusterComponents( + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + MetricRegistry metricRegistry) throws Exception { + synchronized (lock) { + archivedExecutionGraphStore = createSerializableExecutionGraphStore(configuration, rpcService.getScheduledExecutor()); + + dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); + + resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever(); + + final ClusterInformation clusterInformation = new ClusterInformation( + rpcService.getAddress(), + blobServer.getPort()); + + transientBlobCache = new TransientBlobCache( + configuration, + new InetSocketAddress( + clusterInformation.getBlobServerHostname(), + clusterInformation.getBlobServerPort())); + + LeaderGatewayRetriever dispatcherGatewayRetriever = new RpcGatewayRetriever<>( + rpcService, + DispatcherGateway.class, + DispatcherId::new, + 10, + Time.milliseconds(50L)); + + LeaderGatewayRetriever resourceManagerGatewayRetriever = new RpcGatewayRetriever<>( + rpcService, + ResourceManagerGateway.class, + ResourceManagerId::new, + 10, + Time.milliseconds(50L)); + + // TODO: Remove once we have ported the MetricFetcher to the RpcEndpoint + final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem(); + final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); + + webMonitorEndpoint = createRestEndpoint( + configuration, + dispatcherGatewayRetriever, + resourceManagerGatewayRetriever, + transientBlobCache, + rpcService.getExecutor(), + new AkkaQueryServiceRetriever(actorSystem, timeout), + highAvailabilityServices.getWebMonitorLeaderElectionService()); + + LOG.debug("Starting Dispatcher REST endpoint."); + webMonitorEndpoint.start(); + + resourceManager = createResourceManager( + configuration, + ResourceID.generate(), + rpcService, + highAvailabilityServices, + heartbeatServices, + metricRegistry, + this, + clusterInformation, + webMonitorEndpoint.getRestAddress()); + + dispatcher = createDispatcher( + configuration, + rpcService, + highAvailabilityServices, + resourceManager.getSelfGateway(ResourceManagerGateway.class), + blobServer, + heartbeatServices, + metricRegistry, + archivedExecutionGraphStore, + this, + webMonitorEndpoint.getRestAddress()); + + LOG.debug("Starting ResourceManager."); + resourceManager.start(); + resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); + + LOG.debug("Starting Dispatcher."); + dispatcher.start(); + dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever); + } } /** @@ -228,7 +376,7 @@ protected MetricRegistryImpl createMetricRegistry(Configuration configuration) { return new MetricRegistryImpl(MetricRegistryConfiguration.fromConfiguration(configuration)); } - protected void shutDown(boolean cleanupHaData) throws FlinkException { + private void shutDown(boolean cleanupHaData) throws FlinkException { LOG.info("Stopping {}.", getClass().getSimpleName()); Throwable exception = null; @@ -236,11 +384,29 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { synchronized (lock) { try { - stopClusterComponents(cleanupHaData); + stopClusterComponents(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + try { + stopClusterServices(cleanupHaData); } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } + terminationFuture.complete(true); + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the cluster entrypoint.", exception); + } + } + + protected void stopClusterServices(boolean cleanupHaData) throws FlinkException { + Throwable exception = null; + + synchronized (lock) { if (metricRegistry != null) { try { metricRegistry.shutdown(); @@ -276,8 +442,6 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { exception = ExceptionUtils.firstOrSuppressed(t, exception); } } - - terminationFuture.complete(true); } if (exception != null) { @@ -285,6 +449,68 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { } } + protected void stopClusterComponents() throws Exception { + synchronized (lock) { + Throwable exception = null; + + if (webMonitorEndpoint != null) { + webMonitorEndpoint.shutdown(Time.seconds(10L)); + } + + if (dispatcherLeaderRetrievalService != null) { + try { + dispatcherLeaderRetrievalService.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (dispatcher != null) { + try { + dispatcher.shutDown(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (resourceManagerRetrievalService != null) { + try { + resourceManagerRetrievalService.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (resourceManager != null) { + try { + resourceManager.shutDown(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (archivedExecutionGraphStore != null) { + try { + archivedExecutionGraphStore.close(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (transientBlobCache != null) { + try { + transientBlobCache.close(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the session cluster entry point.", exception); + } + } + } + @Override public void onFatalError(Throwable exception) { LOG.error("Fatal error occurred in the cluster entrypoint.", exception); @@ -292,16 +518,74 @@ public void onFatalError(Throwable exception) { System.exit(RUNTIME_FAILURE_RETURN_CODE); } - protected abstract void startClusterComponents( + // -------------------------------------------------- + // Internal methods + // -------------------------------------------------- + + private void shutDownAndTerminate( + int returnCode, + ApplicationStatus applicationStatus, + boolean cleanupHaData) { + + LOG.info("Shut down and terminate {} with return code {} and application status {}.", + getClass().getSimpleName(), + returnCode, + applicationStatus); + + if (isTerminating.compareAndSet(false, true)) { + try { + shutDown(cleanupHaData); + } catch (Throwable t) { + LOG.info("Could not properly shut down cluster entrypoint.", t); + } + + System.exit(returnCode); + } else { + LOG.debug("Concurrent termination call detected. Ignoring termination call with return code {} and application status {}.", + returnCode, + applicationStatus); + } + } + + // -------------------------------------------------- + // Abstract methods + // -------------------------------------------------- + + protected abstract Dispatcher createDispatcher( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, + ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception; + MetricRegistry metricRegistry, + ArchivedExecutionGraphStore archivedExecutionGraphStore, + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress) throws Exception; - protected void stopClusterComponents(boolean cleanupHaData) throws Exception { - } + protected abstract ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + MetricRegistry metricRegistry, + FatalErrorHandler fatalErrorHandler, + ClusterInformation clusterInformation, + @Nullable String webInterfaceUrl) throws Exception; + + protected abstract WebMonitorEndpoint createRestEndpoint( + Configuration configuration, + LeaderGatewayRetriever dispatcherGatewayRetriever, + LeaderGatewayRetriever resourceManagerGatewayRetriever, + TransientBlobService transientBlobService, + Executor executor, + MetricQueryServiceRetriever metricQueryServiceRetriever, + LeaderElectionService leaderElectionService) throws Exception; + + protected abstract ArchivedExecutionGraphStore createSerializableExecutionGraphStore( + Configuration configuration, + ScheduledExecutor scheduledExecutor) throws IOException; protected static ClusterConfiguration parseArguments(String[] args) { ParameterTool parameterTool = ParameterTool.fromArgs(args); @@ -314,4 +598,19 @@ protected static ClusterConfiguration parseArguments(String[] args) { protected static Configuration loadConfiguration(ClusterConfiguration clusterConfiguration) { return GlobalConfiguration.loadConfiguration(clusterConfiguration.getConfigDir()); } + + /** + * Execution mode of the {@link MiniDispatcher}. + */ + public enum ExecutionMode { + /** + * Waits until the job result has been served. + */ + NORMAL, + + /** + * Directly stops after the job has finished. + */ + DETACHED + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java index e03c318f554bb..dc211d8406053 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java @@ -18,52 +18,33 @@ package org.apache.flink.runtime.entrypoint; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.blob.TransientBlobService; -import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.executiongraph.ErrorInfo; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.Dispatcher; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; +import org.apache.flink.runtime.dispatcher.MiniDispatcher; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.jobmaster.JobMasterId; -import org.apache.flink.runtime.jobmaster.JobMasterRestEndpoint; +import org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; -import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; -import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; -import org.apache.flink.util.ConfigurationException; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; - -import akka.actor.ActorSystem; import javax.annotation.Nullable; -import java.net.InetSocketAddress; +import java.io.IOException; import java.util.concurrent.Executor; /** @@ -71,123 +52,24 @@ */ public abstract class JobClusterEntrypoint extends ClusterEntrypoint { - private ResourceManager resourceManager; - - private JobManagerSharedServices jobManagerSharedServices; - - private JobMasterRestEndpoint jobMasterRestEndpoint; - - private LeaderRetrievalService jobMasterRetrievalService; - - private LeaderRetrievalService resourceManagerRetrievalService; - - private TransientBlobCache transientBlobCache; - - private JobManagerRunner jobManagerRunner; - public JobClusterEntrypoint(Configuration configuration) { super(configuration); } @Override - protected void startClusterComponents( + protected MiniDispatcherRestEndpoint createRestEndpoint( Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { - - jobManagerSharedServices = JobManagerSharedServices.fromConfiguration(configuration, blobServer); - - resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever(); - - final LeaderGatewayRetriever jobMasterGatewayRetriever = new RpcGatewayRetriever<>( - rpcService, - JobMasterGateway.class, - JobMasterId::new, - 10, - Time.milliseconds(50L)); - - final LeaderGatewayRetriever resourceManagerGatewayRetriever = new RpcGatewayRetriever<>( - rpcService, - ResourceManagerGateway.class, - ResourceManagerId::new, - 10, - Time.milliseconds(50L)); - - // TODO: Remove once we have ported the MetricFetcher to the RpcEndpoint - final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem(); - final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); - - final ClusterInformation clusterInformation = new ClusterInformation(rpcService.getAddress(), blobServer.getPort()); - - transientBlobCache = new TransientBlobCache( - configuration, - new InetSocketAddress(clusterInformation.getBlobServerHostname(), clusterInformation.getBlobServerPort())); - - jobMasterRestEndpoint = createJobMasterRestEndpoint( - configuration, - jobMasterGatewayRetriever, - resourceManagerGatewayRetriever, - transientBlobCache, - rpcService.getExecutor(), - new AkkaQueryServiceRetriever(actorSystem, timeout), - highAvailabilityServices.getWebMonitorLeaderElectionService()); - - LOG.debug("Starting JobMaster REST endpoint."); - jobMasterRestEndpoint.start(); - - resourceManager = createResourceManager( - configuration, - ResourceID.generate(), - rpcService, - highAvailabilityServices, - heartbeatServices, - metricRegistry, - this, - clusterInformation, - jobMasterRestEndpoint.getRestAddress()); - - jobManagerRunner = createJobManagerRunner( - configuration, - ResourceID.generate(), - rpcService, - highAvailabilityServices, - jobManagerSharedServices, - heartbeatServices, - blobServer, - metricRegistry, - this, - jobMasterRestEndpoint.getRestAddress()); - - LOG.debug("Starting ResourceManager."); - resourceManager.start(); - resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); - - LOG.debug("Starting JobManager."); - jobManagerRunner.start(); - - jobMasterRetrievalService = highAvailabilityServices.getJobManagerLeaderRetriever( - jobManagerRunner.getJobGraph().getJobID(), - jobManagerRunner.getAddress()); - jobMasterRetrievalService.start(jobMasterGatewayRetriever); - } - - protected JobMasterRestEndpoint createJobMasterRestEndpoint( - Configuration configuration, - GatewayRetriever jobMasterGatewayRetriever, - GatewayRetriever resourceManagerGatewayRetriever, + LeaderGatewayRetriever dispatcherGatewayRetriever, + LeaderGatewayRetriever resourceManagerGatewayRetriever, TransientBlobService transientBlobService, Executor executor, MetricQueryServiceRetriever metricQueryServiceRetriever, - LeaderElectionService leaderElectionService) throws ConfigurationException { - + LeaderElectionService leaderElectionService) throws Exception { final RestHandlerConfiguration restHandlerConfiguration = RestHandlerConfiguration.fromConfiguration(configuration); - return new JobMasterRestEndpoint( + return new MiniDispatcherRestEndpoint( RestServerEndpointConfiguration.fromConfiguration(configuration), - jobMasterGatewayRetriever, + dispatcherGatewayRetriever, configuration, restHandlerConfiguration, resourceManagerGatewayRetriever, @@ -196,155 +78,50 @@ protected JobMasterRestEndpoint createJobMasterRestEndpoint( metricQueryServiceRetriever, leaderElectionService, this); + } + @Override + protected ArchivedExecutionGraphStore createSerializableExecutionGraphStore( + Configuration configuration, + ScheduledExecutor scheduledExecutor) throws IOException { + return new MemoryArchivedExecutionGraphStore(); } - protected JobManagerRunner createJobManagerRunner( + @Override + protected Dispatcher createDispatcher( Configuration configuration, - ResourceID resourceId, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - JobManagerSharedServices jobManagerSharedServices, - HeartbeatServices heartbeatServices, + ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, + HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, + ArchivedExecutionGraphStore archivedExecutionGraphStore, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { final JobGraph jobGraph = retrieveJobGraph(configuration); - return new JobManagerRunner( - resourceId, - jobGraph, - configuration, + final String executionModeValue = configuration.getString(EXECUTION_MODE); + + final ExecutionMode executionMode = ExecutionMode.valueOf(executionModeValue); + + return new MiniDispatcher( rpcService, + Dispatcher.DISPATCHER_NAME, + configuration, highAvailabilityServices, - heartbeatServices, + resourceManagerGateway, blobServer, - jobManagerSharedServices, + heartbeatServices, metricRegistry, - new TerminatingOnCompleteActions(jobGraph.getJobID()), + archivedExecutionGraphStore, + Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, fatalErrorHandler, - restAddress); - } - - @Override - protected void stopClusterComponents(boolean cleanupHaData) throws Exception { - Throwable exception = null; - - if (jobMasterRestEndpoint != null) { - jobMasterRestEndpoint.shutdown(Time.seconds(10L)); - } - - if (jobMasterRetrievalService != null) { - try { - jobMasterRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (jobManagerRunner != null) { - try { - jobManagerRunner.shutdown(); - } catch (Throwable t) { - exception = t; - } - } - - if (jobManagerSharedServices != null) { - try { - jobManagerSharedServices.shutdown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (resourceManagerRetrievalService != null) { - try { - resourceManagerRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (resourceManager != null) { - try { - resourceManager.shutDown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (transientBlobCache != null) { - try { - transientBlobCache.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (exception != null) { - throw new FlinkException("Could not properly shut down the job cluster entry point.", exception); - } - } - - private void shutDownAndTerminate( - boolean cleanupHaData, - ApplicationStatus status, - @Nullable String optionalDiagnostics) { - try { - if (resourceManager != null) { - resourceManager.shutDownCluster(status, optionalDiagnostics); - } - - shutDown(cleanupHaData); - } catch (Throwable t) { - LOG.error("Could not properly shut down cluster entrypoint.", t); - } - - System.exit(SUCCESS_RETURN_CODE); + restAddress, + jobGraph, + executionMode); } - protected abstract ResourceManager createResourceManager( - Configuration configuration, - ResourceID resourceId, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, - FatalErrorHandler fatalErrorHandler, - ClusterInformation clusterInformation, - @Nullable String webInterfaceUrl) throws Exception; - protected abstract JobGraph retrieveJobGraph(Configuration configuration) throws FlinkException; - - private final class TerminatingOnCompleteActions implements OnCompletionActions { - - private final JobID jobId; - - private TerminatingOnCompleteActions(JobID jobId) { - this.jobId = Preconditions.checkNotNull(jobId); - } - - @Override - public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { - LOG.info("Job({}) finished.", jobId); - - final ErrorInfo errorInfo = executionGraph.getFailureInfo(); - - if (errorInfo == null) { - shutDownAndTerminate(true, ApplicationStatus.SUCCEEDED, null); - } else { - shutDownAndTerminate(true, ApplicationStatus.FAILED, errorInfo.getExceptionAsString()); - } - } - - @Override - public void jobFinishedByOther() { - LOG.info("Job({}) was finished by another JobManager.", jobId); - - shutDownAndTerminate(false, ApplicationStatus.UNKNOWN, "Job was finished by another master"); - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java index 0f41c9af40a07..764356d036a85 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java @@ -22,48 +22,33 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.blob.TransientBlobService; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.dispatcher.ArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; -import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint; import org.apache.flink.runtime.dispatcher.FileArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; -import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; -import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.guava18.com.google.common.base.Ticker; -import akka.actor.ActorSystem; - import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.net.InetSocketAddress; import java.util.concurrent.Executor; /** @@ -71,110 +56,12 @@ */ public abstract class SessionClusterEntrypoint extends ClusterEntrypoint { - private ResourceManager resourceManager; - - private Dispatcher dispatcher; - - private LeaderRetrievalService dispatcherLeaderRetrievalService; - - private LeaderRetrievalService resourceManagerRetrievalService; - - private DispatcherRestEndpoint dispatcherRestEndpoint; - - private ArchivedExecutionGraphStore archivedExecutionGraphStore; - - private TransientBlobCache transientBlobCache; - public SessionClusterEntrypoint(Configuration configuration) { super(configuration); } @Override - protected void startClusterComponents( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry) throws Exception { - - archivedExecutionGraphStore = createSerializableExecutionGraphStore(configuration, rpcService.getScheduledExecutor()); - - dispatcherLeaderRetrievalService = highAvailabilityServices.getDispatcherLeaderRetriever(); - - resourceManagerRetrievalService = highAvailabilityServices.getResourceManagerLeaderRetriever(); - - LeaderGatewayRetriever dispatcherGatewayRetriever = new RpcGatewayRetriever<>( - rpcService, - DispatcherGateway.class, - DispatcherId::new, - 10, - Time.milliseconds(50L)); - - LeaderGatewayRetriever resourceManagerGatewayRetriever = new RpcGatewayRetriever<>( - rpcService, - ResourceManagerGateway.class, - ResourceManagerId::new, - 10, - Time.milliseconds(50L)); - - // TODO: Remove once we have ported the MetricFetcher to the RpcEndpoint - final ActorSystem actorSystem = ((AkkaRpcService) rpcService).getActorSystem(); - final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); - - final ClusterInformation clusterInformation = new ClusterInformation( - rpcService.getAddress(), - blobServer.getPort()); - - transientBlobCache = new TransientBlobCache( - configuration, - new InetSocketAddress(clusterInformation.getBlobServerHostname(), clusterInformation.getBlobServerPort())); - - dispatcherRestEndpoint = createDispatcherRestEndpoint( - configuration, - dispatcherGatewayRetriever, - resourceManagerGatewayRetriever, - transientBlobCache, - rpcService.getExecutor(), - new AkkaQueryServiceRetriever(actorSystem, timeout), - highAvailabilityServices.getWebMonitorLeaderElectionService()); - - LOG.debug("Starting Dispatcher REST endpoint."); - dispatcherRestEndpoint.start(); - - resourceManager = createResourceManager( - configuration, - ResourceID.generate(), - rpcService, - highAvailabilityServices, - heartbeatServices, - metricRegistry, - this, - clusterInformation, - dispatcherRestEndpoint.getRestAddress()); - - dispatcher = createDispatcher( - configuration, - rpcService, - highAvailabilityServices, - resourceManager.getSelfGateway(ResourceManagerGateway.class), - blobServer, - heartbeatServices, - metricRegistry, - archivedExecutionGraphStore, - this, - dispatcherRestEndpoint.getRestAddress()); - - LOG.debug("Starting ResourceManager."); - resourceManager.start(); - resourceManagerRetrievalService.start(resourceManagerGatewayRetriever); - - LOG.debug("Starting Dispatcher."); - dispatcher.start(); - dispatcherLeaderRetrievalService.start(dispatcherGatewayRetriever); - } - - private ArchivedExecutionGraphStore createSerializableExecutionGraphStore( + protected ArchivedExecutionGraphStore createSerializableExecutionGraphStore( Configuration configuration, ScheduledExecutor scheduledExecutor) throws IOException { final File tmpDir = new File(ConfigurationUtils.parseTempDirectories(configuration)[0]); @@ -191,74 +78,14 @@ private ArchivedExecutionGraphStore createSerializableExecutionGraphStore( } @Override - protected void stopClusterComponents(boolean cleanupHaData) throws Exception { - Throwable exception = null; - - if (dispatcherRestEndpoint != null) { - dispatcherRestEndpoint.shutdown(Time.seconds(10L)); - } - - if (dispatcherLeaderRetrievalService != null) { - try { - dispatcherLeaderRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (dispatcher != null) { - try { - dispatcher.shutDown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (resourceManagerRetrievalService != null) { - try { - resourceManagerRetrievalService.stop(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (resourceManager != null) { - try { - resourceManager.shutDown(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (archivedExecutionGraphStore != null) { - try { - archivedExecutionGraphStore.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (transientBlobCache != null) { - try { - transientBlobCache.close(); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - } - - if (exception != null) { - throw new FlinkException("Could not properly shut down the session cluster entry point.", exception); - } - } - - protected DispatcherRestEndpoint createDispatcherRestEndpoint( - Configuration configuration, - LeaderGatewayRetriever dispatcherGatewayRetriever, - LeaderGatewayRetriever resourceManagerGatewayRetriever, + protected DispatcherRestEndpoint createRestEndpoint( + Configuration configuration, + LeaderGatewayRetriever dispatcherGatewayRetriever, + LeaderGatewayRetriever resourceManagerGatewayRetriever, TransientBlobService transientBlobService, - Executor executor, - MetricQueryServiceRetriever metricQueryServiceRetriever, - LeaderElectionService leaderElectionService) throws Exception { + Executor executor, + MetricQueryServiceRetriever metricQueryServiceRetriever, + LeaderElectionService leaderElectionService) throws Exception { final RestHandlerConfiguration restHandlerConfiguration = RestHandlerConfiguration.fromConfiguration(configuration); @@ -275,6 +102,7 @@ protected DispatcherRestEndpoint createDispatcherRestEndpoint( this); } + @Override protected Dispatcher createDispatcher( Configuration configuration, RpcService rpcService, @@ -298,18 +126,8 @@ protected Dispatcher createDispatcher( heartbeatServices, metricRegistry, archivedExecutionGraphStore, + Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, fatalErrorHandler, restAddress); } - - protected abstract ResourceManager createResourceManager( - Configuration configuration, - ResourceID resourceId, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, - FatalErrorHandler fatalErrorHandler, - ClusterInformation clusterInformation, - @Nullable String webInterfaceUrl) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java similarity index 86% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java index b0fa46d502b88..54ad526dda246 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/MiniDispatcherRestEndpoint.java @@ -20,11 +20,13 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.TransientBlobService; +import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; @@ -32,13 +34,13 @@ import java.util.concurrent.Executor; /** - * REST endpoint for the {@link JobMaster}. + * REST endpoint for the {@link JobClusterEntrypoint}. */ -public class JobMasterRestEndpoint extends WebMonitorEndpoint { +public class MiniDispatcherRestEndpoint extends WebMonitorEndpoint { - public JobMasterRestEndpoint( + public MiniDispatcherRestEndpoint( RestServerEndpointConfiguration endpointConfiguration, - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, Configuration clusterConfiguration, RestHandlerConfiguration restConfiguration, GatewayRetriever resourceManagerRetriever, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandler.java index 1d14563752271..4b5fa8973632b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandler.java @@ -45,7 +45,7 @@ public final class BlobServerPortHandler extends AbstractRestHandler localRestAddress, - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, Time timeout, Map headers) { super(localRestAddress, leaderRetriever, timeout, headers, BlobServerPortHeaders.getInstance()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java index efb162ebe221c..ac0a17b43d02c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java @@ -46,7 +46,7 @@ public final class JobSubmitHandler extends AbstractRestHandler localRestAddress, - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, Time timeout, Map headers) { super(localRestAddress, leaderRetriever, timeout, headers, JobSubmitHeaders.getInstance()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java index 650647b8eb767..5159e120fd482 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java @@ -110,7 +110,7 @@ public class StaticFileServerHandler extends RedirectH private final File rootPath; public StaticFileServerHandler( - GatewayRetriever retriever, + GatewayRetriever retriever, CompletableFuture localJobManagerAddressFuture, Time timeout, File rootPath) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java index dc8d599f70494..1b1c1e5afb5c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java @@ -44,7 +44,7 @@ public final class TaskExecutorRegistrationSuccess extends RegistrationResponse. /** * Create a new {@code TaskExecutorRegistrationSuccess} message. - * + * * @param registrationId The ID that the ResourceManager assigned the registration. * @param resourceManagerResourceId The unique ID that identifies the ResourceManager. * @param heartbeatInterval The interval in which the ResourceManager will heartbeat the TaskExecutor. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java index f8f9164b10da1..c94e278fa9699 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java @@ -580,7 +580,7 @@ private TaskSlot getTaskSlot(AllocationID allocationId) { } private void checkInit() { - Preconditions.checkState(started, "The " + TaskSlotTable.class.getSimpleName() + " has to be started."); + Preconditions.checkState(started, "The %s has to be started.", TaskSlotTable.class.getSimpleName()); } // --------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 19efcbb148346..cee02fe786f2f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -135,7 +135,7 @@ */ public class WebMonitorEndpoint extends RestServerEndpoint implements LeaderContender { - protected final GatewayRetriever leaderRetriever; + protected final GatewayRetriever leaderRetriever; private final Configuration clusterConfiguration; protected final RestHandlerConfiguration restConfiguration; private final GatewayRetriever resourceManagerRetriever; @@ -145,7 +145,7 @@ public class WebMonitorEndpoint extends RestServerEndp private final ExecutionGraphCache executionGraphCache; private final CheckpointStatsCache checkpointStatsCache; - private final MetricFetcher metricFetcher; + private final MetricFetcher metricFetcher; private final LeaderElectionService leaderElectionService; @@ -153,7 +153,7 @@ public class WebMonitorEndpoint extends RestServerEndp public WebMonitorEndpoint( RestServerEndpointConfiguration endpointConfiguration, - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, Configuration clusterConfiguration, RestHandlerConfiguration restConfiguration, GatewayRetriever resourceManagerRetriever, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java index 89779062b31ae..a603340acdd56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorUtils.java @@ -186,7 +186,7 @@ public static WebMonitor startWebRuntimeMonitor( * @throws IOException if we cannot create the StaticFileServerHandler */ public static Optional> tryLoadWebContent( - GatewayRetriever leaderRetriever, + GatewayRetriever leaderRetriever, CompletableFuture restAddressFuture, Time timeout, File tmpDir) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 8fe3c3bf85018..6ea6383a39504 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -71,6 +71,8 @@ import org.junit.rules.TestName; import org.mockito.Mockito; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -345,8 +347,6 @@ public void testJobRecovery() throws Exception { private static class TestingDispatcher extends Dispatcher { - private final JobID expectedJobId; - private final CountDownLatch submitJobLatch = new CountDownLatch(2); /** @@ -372,19 +372,44 @@ private TestingDispatcher( endpointId, configuration, highAvailabilityServices, + highAvailabilityServices.getSubmittedJobGraphStore(), resourceManagerGateway, blobServer, heartbeatServices, metricRegistry, archivedExecutionGraphStore, + new ExpectedJobIdJobManagerRunnerFactory(expectedJobId), fatalErrorHandler, null); + } + + @Override + public CompletableFuture submitJob(final JobGraph jobGraph, final Time timeout) { + final CompletableFuture submitJobFuture = super.submitJob(jobGraph, timeout); + + submitJobFuture.thenAccept(ignored -> submitJobLatch.countDown()); + + return submitJobFuture; + } + + @Override + void recoverJobs() { + if (recoverJobsEnabled.get()) { + super.recoverJobs(); + } + } + } + private static final class ExpectedJobIdJobManagerRunnerFactory implements Dispatcher.JobManagerRunnerFactory { + + private final JobID expectedJobId; + + private ExpectedJobIdJobManagerRunnerFactory(JobID expectedJobId) { this.expectedJobId = expectedJobId; } @Override - protected JobManagerRunner createJobManagerRunner( + public JobManagerRunner createJobManagerRunner( ResourceID resourceId, JobGraph jobGraph, Configuration configuration, @@ -395,10 +420,11 @@ protected JobManagerRunner createJobManagerRunner( JobManagerSharedServices jobManagerSharedServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, - FatalErrorHandler fatalErrorHandler) throws Exception { + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress) throws Exception { assertEquals(expectedJobId, jobGraph.getJobID()); - return new JobManagerRunner( + return Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE.createJobManagerRunner( resourceId, jobGraph, configuration, @@ -410,23 +436,7 @@ protected JobManagerRunner createJobManagerRunner( metricRegistry, onCompleteActions, fatalErrorHandler, - null); - } - - @Override - public CompletableFuture submitJob(final JobGraph jobGraph, final Time timeout) { - final CompletableFuture submitJobFuture = super.submitJob(jobGraph, timeout); - - submitJobFuture.thenAccept(ignored -> submitJobLatch.countDown()); - - return submitJobFuture; - } - - @Override - void recoverJobs() { - if (recoverJobsEnabled.get()) { - super.recoverJobs(); - } + restAddress); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java new file mode 100644 index 0000000000000..bc2478d3e6de4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -0,0 +1,211 @@ +/* + * 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.dispatcher; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.mock; + +/** + * Tests for the {@link MiniDispatcher}. + */ +@Category(Flip6.class) +public class MiniDispatcherTest extends TestLogger { + + private static final JobGraph jobGraph = new JobGraph(); + + private static final Time timeout = Time.seconds(10L); + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static TestingRpcService rpcService; + + private static Configuration configuration; + + private static BlobServer blobServer; + + private MiniDispatcher miniDispatcher; + + private CompletableFuture jobGraphFuture; + + private TestingLeaderElectionService dispatcherLeaderElectionService; + + @BeforeClass + public static void setupClass() throws IOException { + rpcService = new TestingRpcService(); + configuration = new Configuration(); + + configuration.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + + blobServer = new BlobServer(configuration, new VoidBlobStore()); + } + + @Before + public void setup() throws Exception { + dispatcherLeaderElectionService = new TestingLeaderElectionService(); + final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); + final TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); + final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + final ArchivedExecutionGraphStore archivedExecutionGraphStore = new MemoryArchivedExecutionGraphStore(); + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + + highAvailabilityServices.setDispatcherLeaderElectionService(dispatcherLeaderElectionService); + + jobGraphFuture = new CompletableFuture<>(); + final TestingJobManagerRunnerFactory testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(jobGraphFuture); + + miniDispatcher = new MiniDispatcher( + rpcService, + UUID.randomUUID().toString(), + configuration, + highAvailabilityServices, + resourceManagerGateway, + blobServer, + heartbeatServices, + NoOpMetricRegistry.INSTANCE, + archivedExecutionGraphStore, + testingJobManagerRunnerFactory, + testingFatalErrorHandler, + null, + jobGraph, + ClusterEntrypoint.ExecutionMode.DETACHED); + + miniDispatcher.start(); + } + + @After + public void teardown() throws InterruptedException, ExecutionException, TimeoutException { + if (miniDispatcher != null) { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + miniDispatcher = null; + } + } + + @AfterClass + public static void teardownClass() throws IOException { + blobServer.close(); + rpcService.stopService(); + } + + /** + * Tests that the {@link MiniDispatcher} recovers the single job with which it + * was started. + */ + @Test + public void testSingleJobRecovery() throws Exception { + // wait until the Dispatcher is the leader + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + + final JobGraph actualJobGraph = jobGraphFuture.get(); + + assertThat(actualJobGraph.getJobID(), is(jobGraph.getJobID())); + } + + /** + * Tests that in detached mode, the {@link MiniDispatcher} will terminate after the job + * has completed. + */ + @Test + public void testTerminationAfterJobCompletion() throws Exception { + final Dispatcher.DispatcherOnCompleteActions completeActions = miniDispatcher.new DispatcherOnCompleteActions(jobGraph.getJobID()); + + final ArchivedExecutionGraph archivedExecutionGraph = new ArchivedExecutionGraphBuilder() + .setJobID(jobGraph.getJobID()) + .setState(JobStatus.FINISHED) + .build(); + + // wait until the Dispatcher is the leader + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + + // wait until we have submitted the job + jobGraphFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + completeActions.jobReachedGloballyTerminalState(archivedExecutionGraph); + + final CompletableFuture terminationFuture = miniDispatcher.getTerminationFuture(); + + // wait until we terminate + terminationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + + private static final class TestingJobManagerRunnerFactory implements Dispatcher.JobManagerRunnerFactory { + + private final CompletableFuture jobGraphFuture; + + private TestingJobManagerRunnerFactory(CompletableFuture jobGraphFuture) { + this.jobGraphFuture = jobGraphFuture; + } + + @Override + public JobManagerRunner createJobManagerRunner(ResourceID resourceId, JobGraph jobGraph, Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, HeartbeatServices heartbeatServices, BlobServer blobServer, JobManagerSharedServices jobManagerSharedServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { + jobGraphFuture.complete(jobGraph); + + return mock(JobManagerRunner.class); + } + } + +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java index 95a31be0ba639..ec41d8e12b9bb 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java @@ -74,7 +74,10 @@ protected String getYarnJobClusterEntrypoint() { } @Override - public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { + public YarnClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { throw new UnsupportedOperationException("Cannot deploy a per-job cluster yet."); } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index bea90014f6e96..037e086ae8583 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -85,7 +85,10 @@ public void testPerJobMode() throws Exception { jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - ClusterClient clusterClient = flip6YarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph); + ClusterClient clusterClient = flip6YarnClusterDescriptor.deployJobCluster( + clusterSpecification, + jobGraph, + true); clusterClient.shutdown(); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index f015235059786..583ee54748553 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -35,6 +35,7 @@ import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.util.FlinkException; @@ -384,24 +385,13 @@ public ClusterClient deploySessionCluster(ClusterSpecification cl return deployInternal( clusterSpecification, getYarnSessionClusterEntrypoint(), - null); + null, + false); } catch (Exception e) { throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); } } - @Override - public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) throws ClusterDeploymentException { - try { - return deployInternal( - clusterSpecification, - getYarnJobClusterEntrypoint(), - jobGraph); - } catch (Exception e) { - throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); - } - } - @Override public void terminateCluster(ApplicationId applicationId) throws FlinkException { try { @@ -416,12 +406,15 @@ public void terminateCluster(ApplicationId applicationId) throws FlinkException * deployed on YARN. * * @param clusterSpecification Initial cluster specification for the to be deployed Flink cluster - * @param jobGraph A job graph which is deployed with the Flink cluster, null if none + * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. + * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none + * @param detached True if the cluster should be started in detached mode */ protected ClusterClient deployInternal( ClusterSpecification clusterSpecification, String yarnClusterEntrypoint, - @Nullable JobGraph jobGraph) throws Exception { + @Nullable JobGraph jobGraph, + boolean detached) throws Exception { if (UserGroupInformation.isSecurityEnabled()) { // note: UGI::hasKerberosCredentials inaccurately reports false @@ -482,7 +475,14 @@ protected ClusterClient deployInternal( LOG.info("Cluster specification: {}", validClusterSpecification); + final ClusterEntrypoint.ExecutionMode executionMode = detached ? + ClusterEntrypoint.ExecutionMode.DETACHED + : ClusterEntrypoint.ExecutionMode.NORMAL; + + flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); + ApplicationReport report = startAppMaster( + flinkConfiguration, yarnClusterEntrypoint, jobGraph, yarnClient, @@ -635,6 +635,7 @@ private void checkYarnQueues(YarnClient yarnClient) { } public ApplicationReport startAppMaster( + Configuration configuration, String yarnClusterEntrypoint, JobGraph jobGraph, YarnClient yarnClient, @@ -644,7 +645,7 @@ public ApplicationReport startAppMaster( // ------------------ Initialize the file systems ------------------------- try { - org.apache.flink.core.fs.FileSystem.initialize(flinkConfiguration); + org.apache.flink.core.fs.FileSystem.initialize(configuration); } catch (IOException e) { throw new IOException("Error while setting the default " + "filesystem scheme from configuration.", e); @@ -699,16 +700,16 @@ public ApplicationReport startAppMaster( // no user specified cli argument for namespace? if (zkNamespace == null || zkNamespace.isEmpty()) { // namespace defined in config? else use applicationId as default. - zkNamespace = flinkConfiguration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); + zkNamespace = configuration.getString(HighAvailabilityOptions.HA_CLUSTER_ID, String.valueOf(appId)); setZookeeperNamespace(zkNamespace); } - flinkConfiguration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); + configuration.setString(HighAvailabilityOptions.HA_CLUSTER_ID, zkNamespace); - if (HighAvailabilityMode.isHighAvailabilityModeActivated(flinkConfiguration)) { + if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { // activate re-execution of failed applications appContext.setMaxAppAttempts( - flinkConfiguration.getInteger( + configuration.getInteger( YarnConfigOptions.APPLICATION_ATTEMPTS.key(), YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); @@ -716,7 +717,7 @@ public ApplicationReport startAppMaster( } else { // set number of application retries to 1 in the default case appContext.setMaxAppAttempts( - flinkConfiguration.getInteger( + configuration.getInteger( YarnConfigOptions.APPLICATION_ATTEMPTS.key(), 1)); } @@ -797,7 +798,7 @@ public ApplicationReport startAppMaster( // write out configuration file File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); tmpConfigurationFile.deleteOnExit(); - BootstrapTools.writeConfiguration(flinkConfiguration, tmpConfigurationFile); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); Path remotePathConf = setupSingleLocalResource( "flink-conf.yaml", @@ -884,7 +885,7 @@ public ApplicationReport startAppMaster( // setup security tokens Path remotePathKeytab = null; - String keytab = flinkConfiguration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + String keytab = configuration.getString(SecurityOptions.KERBEROS_LOGIN_KEYTAB); if (keytab != null) { LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); remotePathKeytab = setupSingleLocalResource( @@ -916,7 +917,7 @@ public ApplicationReport startAppMaster( // Setup CLASSPATH and environment variables for ApplicationMaster final Map appMasterEnv = new HashMap<>(); // set user specified app master environment variables - appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, flinkConfiguration)); + appMasterEnv.putAll(Utils.getEnvironmentVariables(ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, configuration)); // set Flink app class path appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); @@ -937,7 +938,7 @@ public ApplicationReport startAppMaster( if (remotePathKeytab != null) { appMasterEnv.put(YarnConfigKeys.KEYTAB_PATH, remotePathKeytab.toString()); - String principal = flinkConfiguration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + String principal = configuration.getString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java index e3f96466a7177..461dd555a45e7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java @@ -18,9 +18,12 @@ package org.apache.flink.yarn; +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; @@ -51,6 +54,27 @@ protected String getYarnJobClusterEntrypoint() { return YarnJobClusterEntrypoint.class.getName(); } + @Override + public ClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) throws ClusterDeploymentException { + + // this is required to work with Flip-6 because the slots are allocated + // lazily + jobGraph.setAllowQueuedScheduling(true); + + try { + return deployInternal( + clusterSpecification, + getYarnJobClusterEntrypoint(), + jobGraph, + detached); + } catch (Exception e) { + throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); + } + } + @Override protected ClusterClient createYarnClusterClient( AbstractYarnClusterDescriptor descriptor, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 94673265aff77..a5254a0e3a376 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -50,7 +50,10 @@ protected String getYarnJobClusterEntrypoint() { } @Override - public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { + public YarnClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { throw new UnsupportedOperationException("Cannot deploy a per-job yarn cluster yet."); } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java index 0859f034463b9..8db37406519c2 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/AbstractYarnClusterTest.java @@ -18,9 +18,12 @@ package org.apache.flink.yarn; +import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -137,5 +140,10 @@ protected String getYarnJobClusterEntrypoint() { protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception { throw new UnsupportedOperationException("Not needed for testing"); } + + @Override + public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) throws ClusterDeploymentException { + throw new UnsupportedOperationException("Not needed for testing"); + } } } From e4da50b4105618406a47d75c801a120a71d34cf9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 6 Feb 2018 16:47:28 +0100 Subject: [PATCH 7/8] [FLINK-8609] [flip6] Enable Flip-6 job mode in CliFrontend This commit allows to deploy detached job mode clusters via the CliFrontend. In order to do that, it first extracts the JobGraph from the PackagedProgram and then uses the ClusterDescriptor to deploy the job mode cluster. This closes #5432. --- .../apache/flink/client/cli/CliFrontend.java | 166 ++++++++++++++---- 1 file changed, 130 insertions(+), 36 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index 560bc6f209a4a..23e82bcec8713 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -23,10 +23,13 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.JobWithJars; +import org.apache.flink.client.program.OptimizerPlanEnvironment; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.program.ProgramMissingJobException; @@ -37,6 +40,7 @@ import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.costs.DefaultCostEstimator; @@ -44,9 +48,11 @@ import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.StreamingPlan; import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.JobManagerMessages; @@ -68,6 +74,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.net.InetSocketAddress; +import java.net.URISyntaxException; import java.net.URL; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -113,6 +120,8 @@ public class CliFrontend { private final int defaultParallelism; + private final boolean flip6; + public CliFrontend( Configuration configuration, List> customCommandLines) throws Exception { @@ -135,6 +144,8 @@ public CliFrontend( this.clientTimeout = AkkaUtils.getClientTimeout(this.configuration); this.defaultParallelism = configuration.getInteger(CoreOptions.DEFAULT_PARALLELISM); + + this.flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(configuration.getString(CoreOptions.MODE)); } // -------------------------------------------------------------------------------------------- @@ -195,7 +206,11 @@ protected void run(String[] args) throws Exception { final CustomCommandLine customCommandLine = getActiveCustomCommandLine(commandLine); - runProgram(customCommandLine, commandLine, runOptions, program); + try { + runProgram(customCommandLine, commandLine, runOptions, program); + } finally { + program.deleteExtractedLibraries(); + } } private void runProgram( @@ -210,51 +225,72 @@ private void runProgram( final ClusterClient client; - if (clusterId != null) { - client = clusterDescriptor.retrieve(clusterId); - } else { - final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine); - client = clusterDescriptor.deploySessionCluster(clusterSpecification); - } + // directly deploy the job if the cluster is started in job mode and detached + if (flip6 && clusterId == null && runOptions.getDetachedMode()) { + int parallelism = runOptions.getParallelism() == -1 ? defaultParallelism : runOptions.getParallelism(); - try { - client.setPrintStatusDuringExecution(runOptions.getStdoutLogging()); - client.setDetached(runOptions.getDetachedMode()); - LOG.debug("Client slots is set to {}", client.getMaxSlots()); - - LOG.debug(runOptions.getSavepointRestoreSettings().toString()); - - int userParallelism = runOptions.getParallelism(); - LOG.debug("User parallelism is set to {}", userParallelism); - if (client.getMaxSlots() != -1 && userParallelism == -1) { - logAndSysout("Using the parallelism provided by the remote cluster (" - + client.getMaxSlots() + "). " - + "To use another parallelism, set it at the ./bin/flink client."); - userParallelism = client.getMaxSlots(); - } else if (ExecutionConfig.PARALLELISM_DEFAULT == userParallelism) { - userParallelism = defaultParallelism; - } + final JobGraph jobGraph = createJobGraph(configuration, program, parallelism); - executeProgram(program, client, userParallelism); - } finally { - if (clusterId == null && !client.isDetached()) { - // terminate the cluster only if we have started it before and if it's not detached - try { - clusterDescriptor.terminateCluster(client.getClusterId()); - } catch (FlinkException e) { - LOG.info("Could not properly terminate the Flink cluster.", e); - } - } + final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine); + client = clusterDescriptor.deployJobCluster( + clusterSpecification, + jobGraph, + runOptions.getDetachedMode()); + + logAndSysout("Job has been submitted with JobID " + jobGraph.getJobID()); try { client.shutdown(); } catch (Exception e) { LOG.info("Could not properly shut down the client.", e); } + } else { + if (clusterId != null) { + client = clusterDescriptor.retrieve(clusterId); + } else { + // also in job mode we have to deploy a session cluster because the job + // might consist of multiple parts (e.g. when using collect) + final ClusterSpecification clusterSpecification = customCommandLine.getClusterSpecification(commandLine); + client = clusterDescriptor.deploySessionCluster(clusterSpecification); + } + + try { + client.setPrintStatusDuringExecution(runOptions.getStdoutLogging()); + client.setDetached(runOptions.getDetachedMode()); + LOG.debug("Client slots is set to {}", client.getMaxSlots()); + + LOG.debug("{}", runOptions.getSavepointRestoreSettings()); + + int userParallelism = runOptions.getParallelism(); + LOG.debug("User parallelism is set to {}", userParallelism); + if (client.getMaxSlots() != -1 && userParallelism == -1) { + logAndSysout("Using the parallelism provided by the remote cluster (" + + client.getMaxSlots() + "). " + + "To use another parallelism, set it at the ./bin/flink client."); + userParallelism = client.getMaxSlots(); + } else if (ExecutionConfig.PARALLELISM_DEFAULT == userParallelism) { + userParallelism = defaultParallelism; + } + + executeProgram(program, client, userParallelism); + } finally { + if (clusterId == null && !client.isDetached()) { + // terminate the cluster only if we have started it before and if it's not detached + try { + clusterDescriptor.terminateCluster(client.getClusterId()); + } catch (FlinkException e) { + LOG.info("Could not properly terminate the Flink cluster.", e); + } + } + + try { + client.shutdown(); + } catch (Exception e) { + LOG.info("Could not properly shut down the client.", e); + } + } } } finally { - program.deleteExtractedLibraries(); - try { clusterDescriptor.close(); } catch (Exception e) { @@ -1081,6 +1117,64 @@ public static List> loadCustomCommandLines(Configuration co return customCommandLines; } + /** + * Creates a {@link JobGraph} from the given {@link PackagedProgram}. + * + * @param configuration to use for the optimizer and job graph generator + * @param packagedProgram to extract the JobGraph from + * @param defaultParallelism for the JobGraph + * @return JobGraph extracted from the PackagedProgram + * @throws ProgramInvocationException if the JobGraph generation failed + */ + private static JobGraph createJobGraph(Configuration configuration, PackagedProgram packagedProgram, int defaultParallelism) throws ProgramInvocationException { + Thread.currentThread().setContextClassLoader(packagedProgram.getUserCodeClassLoader()); + final Optimizer optimizer = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), configuration); + final FlinkPlan flinkPlan; + + if (packagedProgram.isUsingProgramEntryPoint()) { + + final JobWithJars jobWithJars = packagedProgram.getPlanWithJars(); + + final Plan plan = jobWithJars.getPlan(); + + if (plan.getDefaultParallelism() <= 0) { + plan.setDefaultParallelism(defaultParallelism); + } + + flinkPlan = optimizer.compile(jobWithJars.getPlan()); + } else if (packagedProgram.isUsingInteractiveMode()) { + final OptimizerPlanEnvironment optimizerPlanEnvironment = new OptimizerPlanEnvironment(optimizer); + + optimizerPlanEnvironment.setParallelism(defaultParallelism); + + flinkPlan = optimizerPlanEnvironment.getOptimizedPlan(packagedProgram); + } else { + throw new ProgramInvocationException("PackagedProgram does not have a valid invocation mode."); + } + + final JobGraph jobGraph; + + if (flinkPlan instanceof StreamingPlan) { + jobGraph = ((StreamingPlan) flinkPlan).getJobGraph(); + jobGraph.setSavepointRestoreSettings(packagedProgram.getSavepointSettings()); + } else { + final JobGraphGenerator jobGraphGenerator = new JobGraphGenerator(configuration); + jobGraph = jobGraphGenerator.compileJobGraph((OptimizedPlan) flinkPlan); + } + + for (URL url : packagedProgram.getAllLibraries()) { + try { + jobGraph.addJar(new Path(url.toURI())); + } catch (URISyntaxException e) { + throw new ProgramInvocationException("Invalid URL for jar file: " + url + '.', e); + } + } + + jobGraph.setClasspaths(packagedProgram.getClasspaths()); + + return jobGraph; + } + // -------------------------------------------------------------------------------------------- // Custom command-line // -------------------------------------------------------------------------------------------- From d1120843d93d3ce3d7bf6ca8c9a39f1289df8c82 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 7 Feb 2018 17:00:40 +0100 Subject: [PATCH 8/8] [FLINK-8613] [flip6] [yarn] Return excess containers Upon notification of newly allocated containers, the YarnResourceManager will only accept as many containers as there are pending container requests. All excess containers will be returned. This closes #5436. --- .../flink/yarn/YarnResourceManager.java | 58 ++++++++++++------- 1 file changed, 37 insertions(+), 21 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 8104d495e8c3c..305a2f575f75c 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -325,26 +325,43 @@ public void onContainersCompleted(List list) { @Override public void onContainersAllocated(List containers) { for (Container container : containers) { - numPendingContainerRequests = Math.max(0, numPendingContainerRequests - 1); - log.info("Received new container: {} - Remaining pending container requests: {}", - container.getId(), numPendingContainerRequests); - final String containerIdStr = container.getId().toString(); - workerNodeMap.put(new ResourceID(containerIdStr), - new YarnWorkerNode(container)); - try { - /** Context information used to start a TaskExecutor Java process */ - ContainerLaunchContext taskExecutorLaunchContext = - createTaskExecutorLaunchContext( - container.getResource(), containerIdStr, container.getNodeId().getHost()); - nodeManagerClient.startContainer(container, taskExecutorLaunchContext); - } - catch (Throwable t) { - // failed to launch the container, will release the failed one and ask for a new one - log.error("Could not start TaskManager in container {},", container, t); + log.info( + "Received new container: {} - Remaining pending container requests: {}", + container.getId(), + numPendingContainerRequests); + + if (numPendingContainerRequests > 0) { + numPendingContainerRequests--; + + final String containerIdStr = container.getId().toString(); + + workerNodeMap.put(new ResourceID(containerIdStr), new YarnWorkerNode(container)); + + try { + // Context information used to start a TaskExecutor Java process + ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorLaunchContext( + container.getResource(), + containerIdStr, + container.getNodeId().getHost()); + + nodeManagerClient.startContainer(container, taskExecutorLaunchContext); + } catch (Throwable t) { + log.error("Could not start TaskManager in container {}.", container.getId(), t); + + // release the failed container + resourceManagerClient.releaseAssignedContainer(container.getId()); + // and ask for a new one + requestYarnContainer(container.getResource(), container.getPriority()); + } + } else { + // return the excessive containers + log.info("Returning excess container {}.", container.getId()); resourceManagerClient.releaseAssignedContainer(container.getId()); - requestYarnContainer(container.getResource(), container.getPriority()); } } + + // if we are waiting for no further containers, we can go to the + // regular heartbeat interval if (numPendingContainerRequests <= 0) { resourceManagerClient.setHeartbeatInterval(yarnHeartbeatIntervalMillis); } @@ -403,14 +420,13 @@ private static Tuple2 parseHostPort(String address) { } private void requestYarnContainer(Resource resource, Priority priority) { - resourceManagerClient.addContainerRequest( - new AMRMClient.ContainerRequest(resource, null, null, priority)); + resourceManagerClient.addContainerRequest(new AMRMClient.ContainerRequest(resource, null, null, priority)); + // make sure we transmit the request fast and receive fast news of granted allocations resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS); numPendingContainerRequests++; - log.info("Requesting new TaskManager container pending requests: {}", - numPendingContainerRequests); + log.info("Requesting new TaskManager container pending requests: {}", numPendingContainerRequests); } private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource, String containerId, String host)