diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java index b58b1b7a9d4a8..f97395c9783ee 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/PerJobMiniClusterFactoryTest.java @@ -113,23 +113,6 @@ public void testJobClientSavepoint() throws Exception { () -> jobClient.stopWithSavepoint(true, null).get()); } - @Test - public void testSubmissionError() throws Exception { - PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); - - // JobGraph is not a valid job - - JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); - - assertThrows( - "Could not instantiate JobManager", - ExecutionException.class, - () -> - perJobMiniClusterFactory - .submitJob(jobGraph, ClassLoader.getSystemClassLoader()) - .get()); - } - @Test public void testMultipleExecutions() throws Exception { PerJobMiniClusterFactory perJobMiniClusterFactory = initializeMiniCluster(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobStatus.java b/flink-core/src/main/java/org/apache/flink/api/common/JobStatus.java index 6bd2011e0c648..5726899b7cd7b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobStatus.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobStatus.java @@ -24,8 +24,8 @@ @PublicEvolving public enum JobStatus { /** - * The job has been received by the Dispatcher, and is waiting for the job manager to be - * created. + * The job has been received by the Dispatcher, and is waiting for the job manager to receive + * leadership and to be created. */ INITIALIZING(TerminalState.NON_TERMINAL), 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 401c4d127edd3..7e2ff220f9492 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 @@ -28,7 +28,6 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; -import org.apache.flink.runtime.client.JobInitializationException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -43,6 +42,7 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; @@ -116,7 +116,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint runningJobs; + private final Map runningJobs; private final Collection recoveredJobs; @@ -134,7 +134,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint> dispatcherJobTerminationFutures; + private final Map> jobManagerRunnerTerminationFutures; protected final CompletableFuture shutDownFuture; @@ -182,7 +182,7 @@ public Dispatcher( this.jobManagerRunnerFactory = dispatcherServices.getJobManagerRunnerFactory(); - this.dispatcherJobTerminationFutures = new HashMap<>(2); + this.jobManagerRunnerTerminationFutures = new HashMap<>(2); this.shutDownFuture = new CompletableFuture<>(); @@ -393,36 +393,30 @@ private void persistAndRunJob(JobGraph jobGraph) throws Exception { runJob(jobGraph, ExecutionType.SUBMISSION); } - private void runJob(JobGraph jobGraph, ExecutionType executionType) { + private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Exception { Preconditions.checkState(!runningJobs.containsKey(jobGraph.getJobID())); long initializationTimestamp = System.currentTimeMillis(); - CompletableFuture jobManagerRunnerFuture = + JobManagerRunner jobManagerRunner = createJobManagerRunner(jobGraph, initializationTimestamp); - DispatcherJob dispatcherJob = - DispatcherJob.createFor( - jobManagerRunnerFuture, - jobGraph.getJobID(), - jobGraph.getName(), - initializationTimestamp); - runningJobs.put(jobGraph.getJobID(), dispatcherJob); + runningJobs.put(jobGraph.getJobID(), jobManagerRunner); final JobID jobId = jobGraph.getJobID(); final CompletableFuture cleanupJobStateFuture = - dispatcherJob + jobManagerRunner .getResultFuture() .handleAsync( - (dispatcherJobResult, throwable) -> { + (jobManagerRunnerResult, throwable) -> { Preconditions.checkState( - runningJobs.get(jobId) == dispatcherJob, - "The job entry in runningJobs must be bound to the lifetime of the DispatcherJob."); + runningJobs.get(jobId) == jobManagerRunner, + "The job entry in runningJobs must be bound to the lifetime of the JobManagerRunner."); - if (dispatcherJobResult != null) { - return handleDispatcherJobResult( - jobId, dispatcherJobResult, executionType); + if (jobManagerRunnerResult != null) { + return handleJobManagerRunnerResult( + jobManagerRunnerResult, executionType); } else { - return dispatcherJobFailed(jobId, throwable); + return jobManagerRunnerFailed(jobId, throwable); } }, getMainThreadExecutor()); @@ -433,16 +427,22 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) { .thenCompose(Function.identity()); FutureUtils.assertNoException(jobTerminationFuture); - registerDispatcherJobTerminationFuture(jobId, jobTerminationFuture); + registerJobManagerRunnerTerminationFuture(jobId, jobTerminationFuture); } - private CleanupJobState handleDispatcherJobResult( - JobID jobId, DispatcherJobResult dispatcherJobResult, ExecutionType executionType) { - if (dispatcherJobResult.isInitializationFailure() - && executionType == ExecutionType.RECOVERY) { - return dispatcherJobFailed(jobId, dispatcherJobResult.getInitializationFailure()); + private CleanupJobState handleJobManagerRunnerResult( + JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { + if (jobManagerRunnerResult.isInitializationFailure()) { + if (executionType == ExecutionType.RECOVERY) { + return jobManagerRunnerFailed( + jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), + jobManagerRunnerResult.getInitializationFailure()); + } else { + return jobReachedGloballyTerminalState( + jobManagerRunnerResult.getExecutionGraphInfo()); + } } else { - return jobReachedGloballyTerminalState(dispatcherJobResult.getExecutionGraphInfo()); + return jobReachedGloballyTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); } } @@ -457,7 +457,7 @@ enum CleanupJobState { } } - private CleanupJobState dispatcherJobFailed(JobID jobId, Throwable throwable) { + private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) { if (throwable instanceof JobNotFinishedException) { jobNotFinished(jobId); } else { @@ -467,36 +467,23 @@ private CleanupJobState dispatcherJobFailed(JobID jobId, Throwable throwable) { return CleanupJobState.LOCAL; } - CompletableFuture createJobManagerRunner( - JobGraph jobGraph, long initializationTimestamp) { + JobManagerRunner createJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) + throws Exception { final RpcService rpcService = getRpcService(); - return CompletableFuture.supplyAsync( - () -> { - try { - JobManagerRunner runner = - jobManagerRunnerFactory.createJobManagerRunner( - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - jobManagerSharedServices, - new DefaultJobManagerJobMetricGroupFactory( - jobManagerMetricGroup), - fatalErrorHandler, - initializationTimestamp); - runner.start(); - return runner; - } catch (Exception e) { - throw new CompletionException( - new JobInitializationException( - jobGraph.getJobID(), - "Could not instantiate JobManager.", - e)); - } - }, - ioExecutor); // do not use main thread executor. Otherwise, Dispatcher is blocked on - // JobManager creation + + JobManagerRunner runner = + jobManagerRunnerFactory.createJobManagerRunner( + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + jobManagerSharedServices, + new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), + fatalErrorHandler, + initializationTimestamp); + runner.start(); + return runner; } @Override @@ -531,7 +518,7 @@ public CompletableFuture disposeSavepoint(String savepointPath, Tim @Override public CompletableFuture cancelJob(JobID jobId, Time timeout) { - Optional maybeJob = getDispatcherJob(jobId); + Optional maybeJob = getJobManagerRunner(jobId); return maybeJob.map(job -> job.cancel(timeout)) .orElseGet( () -> { @@ -550,7 +537,7 @@ public CompletableFuture requestClusterOverview(Time timeout) { final List>> optionalJobInformation = queryJobMastersForInformation( - dispatcherJob -> dispatcherJob.requestJobStatus(timeout)); + jobManagerRunner -> jobManagerRunner.requestJobStatus(timeout)); CompletableFuture>> allOptionalJobsFuture = FutureUtils.combineAll(optionalJobInformation); @@ -572,7 +559,8 @@ public CompletableFuture requestClusterOverview(Time timeout) { @Override public CompletableFuture requestMultipleJobDetails(Time timeout) { List>> individualOptionalJobDetails = - queryJobMastersForInformation(dj -> dj.requestJobDetails(timeout)); + queryJobMastersForInformation( + jobManagerRunner -> jobManagerRunner.requestJobDetails(timeout)); CompletableFuture>> optionalCombinedJobDetails = FutureUtils.combineAll(individualOptionalJobDetails); @@ -597,7 +585,7 @@ public CompletableFuture requestMultipleJobDetails(Time tim @Override public CompletableFuture requestJobStatus(JobID jobId, Time timeout) { - Optional maybeJob = getDispatcherJob(jobId); + Optional maybeJob = getJobManagerRunner(jobId); return maybeJob.map(job -> job.requestJobStatus(timeout)) .orElseGet( () -> { @@ -628,7 +616,7 @@ public CompletableFuture requestExecutionGraphInfo( return executionGraphInfo; } }; - Optional maybeJob = getDispatcherJob(jobId); + Optional maybeJob = getJobManagerRunner(jobId); return maybeJob.map(job -> job.requestJob(timeout)) .orElse(FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId))) .exceptionally(checkExecutionGraphStoreOnException); @@ -636,7 +624,7 @@ public CompletableFuture requestExecutionGraphInfo( @Override public CompletableFuture requestJobResult(JobID jobId, Time timeout) { - DispatcherJob job = runningJobs.get(jobId); + JobManagerRunner job = runningJobs.get(jobId); if (job == null) { final ExecutionGraphInfo executionGraphInfo = executionGraphInfoStore.get(jobId); @@ -650,9 +638,9 @@ public CompletableFuture requestJobResult(JobID jobId, Time timeout) } else { return job.getResultFuture() .thenApply( - dispatcherJobResult -> + jobManagerRunnerResult -> JobResult.createFrom( - dispatcherJobResult + jobManagerRunnerResult .getExecutionGraphInfo() .getArchivedExecutionGraph())); } @@ -728,28 +716,28 @@ public CompletableFuture deliverCoordinationRequestToCoord operatorId, serializedRequest, timeout)); } - private void registerDispatcherJobTerminationFuture( - JobID jobId, CompletableFuture dispatcherJobTerminationFuture) { - Preconditions.checkState(!dispatcherJobTerminationFutures.containsKey(jobId)); - dispatcherJobTerminationFutures.put(jobId, dispatcherJobTerminationFuture); + private void registerJobManagerRunnerTerminationFuture( + JobID jobId, CompletableFuture jobManagerRunnerTerminationFuture) { + Preconditions.checkState(!jobManagerRunnerTerminationFutures.containsKey(jobId)); + jobManagerRunnerTerminationFutures.put(jobId, jobManagerRunnerTerminationFuture); // clean up the pending termination future - dispatcherJobTerminationFuture.thenRunAsync( + jobManagerRunnerTerminationFuture.thenRunAsync( () -> { final CompletableFuture terminationFuture = - dispatcherJobTerminationFutures.remove(jobId); + jobManagerRunnerTerminationFutures.remove(jobId); //noinspection ObjectEquality if (terminationFuture != null - && terminationFuture != dispatcherJobTerminationFuture) { - dispatcherJobTerminationFutures.put(jobId, terminationFuture); + && terminationFuture != jobManagerRunnerTerminationFuture) { + jobManagerRunnerTerminationFutures.put(jobId, terminationFuture); } }, getMainThreadExecutor()); } private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJobState) { - final DispatcherJob job = checkNotNull(runningJobs.remove(jobId)); + final JobManagerRunner job = checkNotNull(runningJobs.remove(jobId)); final CompletableFuture jobTerminationFuture = job.closeAsync(); @@ -796,7 +784,7 @@ private void cleanUpJobData(JobID jobId, boolean cleanupHA) { blobServer.cleanupJob(jobId, cleanupHABlobs); } - /** Terminate all currently running {@link DispatcherJob}s. */ + /** Terminate all currently running {@link JobManagerRunner}s. */ private void terminateRunningJobs() { log.info("Stopping all currently running jobs of dispatcher {}.", getAddress()); @@ -808,16 +796,17 @@ private void terminateRunningJobs() { } private void terminateJob(JobID jobId) { - final DispatcherJob dispatcherJob = runningJobs.get(jobId); + final JobManagerRunner jobManagerRunner = runningJobs.get(jobId); - if (dispatcherJob != null) { - dispatcherJob.closeAsync(); + if (jobManagerRunner != null) { + jobManagerRunner.closeAsync(); } } private CompletableFuture terminateRunningJobsAndGetTerminationFuture() { terminateRunningJobs(); - final Collection> values = dispatcherJobTerminationFutures.values(); + final Collection> values = + jobManagerRunnerTerminationFutures.values(); return FutureUtils.completeAll(values); } @@ -884,10 +873,11 @@ private void jobMasterFailed(JobID jobId, Throwable cause) { /** Ensures that the JobMasterGateway is available. */ private CompletableFuture getJobMasterGateway(JobID jobId) { - DispatcherJob job = runningJobs.get(jobId); + JobManagerRunner job = runningJobs.get(jobId); if (job == null) { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } + if (!job.isInitialized()) { return FutureUtils.completedExceptionally( new UnavailableDispatcherOperationException( @@ -906,7 +896,7 @@ private CompletableFuture getResourceManagerGateway() { return resourceManagerGatewayRetriever.getFuture(); } - private Optional getDispatcherJob(JobID jobId) { + private Optional getJobManagerRunner(JobID jobId) { return Optional.ofNullable(runningJobs.get(jobId)); } @@ -926,12 +916,12 @@ private List flattenOptionalCollection(Collection> optionalCo @Nonnull private List>> queryJobMastersForInformation( - Function> queryFunction) { + Function> queryFunction) { List>> optionalJobInformation = new ArrayList<>(runningJobs.size()); - for (DispatcherJob job : runningJobs.values()) { + for (JobManagerRunner job : runningJobs.values()) { final CompletableFuture> queryResult = queryFunction .apply(job) @@ -958,7 +948,7 @@ private CompletableFuture waitForTerminatingJob( return jobManagerTerminationFuture.thenAcceptAsync( FunctionUtils.uncheckedConsumer( (ignored) -> { - dispatcherJobTerminationFutures.remove(jobId); + jobManagerRunnerTerminationFutures.remove(jobId); action.accept(jobGraph); }), getMainThreadExecutor()); @@ -970,7 +960,7 @@ CompletableFuture getJobTerminationFuture(JobID jobId) { new DispatcherException( String.format("Job with job id %s is still running.", jobId))); } else { - return dispatcherJobTerminationFutures.getOrDefault( + return jobManagerRunnerTerminationFutures.getOrDefault( jobId, CompletableFuture.completedFuture(null)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java deleted file mode 100644 index 7243303a0a851..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJob.java +++ /dev/null @@ -1,284 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.jobmaster.JobNotFinishedException; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.webmonitor.JobDetails; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.util.AutoCloseableAsync; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.Preconditions; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; - -import java.util.concurrent.CompletableFuture; - -/** Abstraction used by the {@link Dispatcher} to manage jobs. */ -public final class DispatcherJob implements AutoCloseableAsync { - - private final Logger log = LoggerFactory.getLogger(DispatcherJob.class); - - private final CompletableFuture jobManagerRunnerFuture; - private final CompletableFuture jobResultFuture; - private final CompletableFuture terminationFuture = new CompletableFuture<>(); - - private final long initializationTimestamp; - private final JobID jobId; - private final String jobName; - - private final Object lock = new Object(); - - // internal field to track job status during initialization. Is not updated anymore after - // job is initialized, cancelled or failed. - @GuardedBy("lock") - private DispatcherJobStatus jobStatus = DispatcherJobStatus.INITIALIZING; - - private enum DispatcherJobStatus { - // We are waiting for the JobManagerRunner to be initialized - INITIALIZING(JobStatus.INITIALIZING), - // JobManagerRunner is initialized - JOB_MANAGER_RUNNER_INITIALIZED(null), - // waiting for cancellation. We stay in this status until the job result future completed, - // then we consider the JobManager to be initialized. - CANCELLING(JobStatus.CANCELLING); - - @Nullable private final JobStatus jobStatus; - - DispatcherJobStatus(JobStatus jobStatus) { - this.jobStatus = jobStatus; - } - - public JobStatus asJobStatus() { - if (jobStatus == null) { - throw new IllegalStateException("This state is not defined as a 'JobStatus'"); - } - return jobStatus; - } - } - - static DispatcherJob createFor( - CompletableFuture jobManagerRunnerFuture, - JobID jobId, - String jobName, - long initializationTimestamp) { - return new DispatcherJob(jobManagerRunnerFuture, jobId, jobName, initializationTimestamp); - } - - private DispatcherJob( - CompletableFuture jobManagerRunnerFuture, - JobID jobId, - String jobName, - long initializationTimestamp) { - this.jobManagerRunnerFuture = jobManagerRunnerFuture; - this.jobId = jobId; - this.jobName = jobName; - this.initializationTimestamp = initializationTimestamp; - this.jobResultFuture = new CompletableFuture<>(); - - FutureUtils.assertNoException( - this.jobManagerRunnerFuture.handle( - (jobManagerRunner, throwable) -> { - // JM has been initialized, or the initialization failed - synchronized (lock) { - jobStatus = DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED; - if (throwable == null) { // initialization succeeded - // Forward result future - jobManagerRunner - .getResultFuture() - .whenComplete( - (jobManagerRunnerResult, resultThrowable) -> { - if (jobManagerRunnerResult != null) { - handleJobManagerRunnerResult( - jobManagerRunnerResult); - } else { - jobResultFuture.completeExceptionally( - ExceptionUtils - .stripCompletionException( - resultThrowable)); - } - }); - } else { // failure during initialization - handleInitializationFailure( - ExceptionUtils.stripCompletionException(throwable)); - } - } - return null; - })); - } - - private void handleJobManagerRunnerResult(JobManagerRunnerResult jobManagerRunnerResult) { - if (jobManagerRunnerResult.isSuccess()) { - jobResultFuture.complete( - DispatcherJobResult.forSuccess(jobManagerRunnerResult.getExecutionGraphInfo())); - } else if (jobManagerRunnerResult.isJobNotFinished()) { - jobResultFuture.completeExceptionally(new JobNotFinishedException(jobId)); - } else if (jobManagerRunnerResult.isInitializationFailure()) { - handleInitializationFailure(jobManagerRunnerResult.getInitializationFailure()); - } - } - - private void handleInitializationFailure(Throwable initializationFailure) { - ArchivedExecutionGraph archivedExecutionGraph = - ArchivedExecutionGraph.createFromInitializingJob( - jobId, - jobName, - JobStatus.FAILED, - initializationFailure, - initializationTimestamp); - jobResultFuture.complete( - DispatcherJobResult.forInitializationFailure( - new ExecutionGraphInfo(archivedExecutionGraph), initializationFailure)); - } - - public CompletableFuture getResultFuture() { - return jobResultFuture; - } - - public CompletableFuture requestJobDetails(Time timeout) { - return requestJob(timeout) - .thenApply( - executionGraphInfo -> { - synchronized (lock) { - return JobDetails.createDetailsForJob( - executionGraphInfo.getArchivedExecutionGraph()); - } - }); - } - - /** - * Cancel job. A cancellation will be scheduled if the initialization is not completed. The - * returned future will complete exceptionally if the JobManagerRunner initialization failed. - */ - public CompletableFuture cancel(Time timeout) { - synchronized (lock) { - if (isInitialized()) { - return getJobMasterGateway() - .thenCompose(jobMasterGateway -> jobMasterGateway.cancel(timeout)); - } else { - log.info( - "Cancellation during initialization requested for job {}. Job will be cancelled once JobManager has been initialized.", - jobId); - - // cancel job - CompletableFuture cancelFuture = - jobManagerRunnerFuture - .thenCompose(JobManagerRunner::getJobMasterGateway) - .thenCompose(jobMasterGateway -> jobMasterGateway.cancel(timeout)); - cancelFuture.whenComplete( - (ignored, cancelThrowable) -> { - if (cancelThrowable != null) { - log.warn("Cancellation of job {} failed", jobId, cancelThrowable); - } - }); - jobStatus = DispatcherJobStatus.CANCELLING; - return cancelFuture; - } - } - } - - public CompletableFuture requestJobStatus(Time timeout) { - return requestJob(timeout) - .thenApply( - executionGraphInfo -> - executionGraphInfo.getArchivedExecutionGraph().getState()); - } - - /** Returns a future completing to the ExecutionGraphInfo of the job. */ - public CompletableFuture requestJob(Time timeout) { - synchronized (lock) { - if (isInitialized()) { - if (jobResultFuture.isDone()) { // job is not running anymore - return jobResultFuture.thenApply(DispatcherJobResult::getExecutionGraphInfo); - } - // job is still running - return getJobMasterGateway() - .thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout)); - } else { - Preconditions.checkState( - this.jobStatus == DispatcherJobStatus.INITIALIZING - || jobStatus == DispatcherJobStatus.CANCELLING); - return CompletableFuture.completedFuture( - new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( - jobId, - jobName, - jobStatus.asJobStatus(), - null, - initializationTimestamp))); - } - } - } - - /** - * The job is initialized once the JobManager runner has been initialized. It is also - * initialized if the runner initialization failed, or of it has been canceled (and the - * cancellation is complete). - */ - public boolean isInitialized() { - synchronized (lock) { - return jobStatus == DispatcherJobStatus.JOB_MANAGER_RUNNER_INITIALIZED; - } - } - - /** - * Returns the {@link JobMasterGateway} from the JobManagerRunner. - * - * @return the {@link JobMasterGateway}. The future will complete exceptionally if the - * JobManagerRunner initialization failed. - * @throws IllegalStateException is thrown if the job is not initialized - */ - public CompletableFuture getJobMasterGateway() { - Preconditions.checkState( - isInitialized(), "JobMaster Gateway is not available during initialization"); - return jobManagerRunnerFuture.thenCompose(JobManagerRunner::getJobMasterGateway); - } - - @Override - public CompletableFuture closeAsync() { - FutureUtils.assertNoException( - jobManagerRunnerFuture.handle( - (runner, throwable) -> { - if (throwable == null) { - // init was successful: close jobManager runner. - CompletableFuture jobManagerRunnerClose = - jobManagerRunnerFuture.thenCompose( - AutoCloseableAsync::closeAsync); - FutureUtils.forward(jobManagerRunnerClose, terminationFuture); - } else { - // initialization has failed: complete termination. - terminationFuture.complete(null); - } - return null; - })); - return terminationFuture; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java deleted file mode 100644 index a5d93da3a087c..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherJobResult.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.util.Preconditions; - -import javax.annotation.Nullable; - -/** - * Container for returning the {@link ExecutionGraphInfo} and a flag whether the initialization has - * failed. For initialization failures, the throwable is also attached, to avoid deserializing it - * from the {@link ArchivedExecutionGraph}. - */ -final class DispatcherJobResult { - - private final ExecutionGraphInfo executionGraphInfo; - - // if the throwable field is set, the job failed during initialization. - @Nullable private final Throwable initializationFailure; - - private DispatcherJobResult( - ExecutionGraphInfo executionGraphInfo, @Nullable Throwable throwable) { - this.executionGraphInfo = executionGraphInfo; - this.initializationFailure = throwable; - } - - public boolean isInitializationFailure() { - return initializationFailure != null; - } - - public ExecutionGraphInfo getExecutionGraphInfo() { - return executionGraphInfo; - } - - /** @throws IllegalStateException if this DispatcherJobResult is a successful initialization. */ - public Throwable getInitializationFailure() { - Preconditions.checkState( - isInitializationFailure(), - "This DispatcherJobResult does not represent a failed initialization."); - return initializationFailure; - } - - public static DispatcherJobResult forInitializationFailure( - ExecutionGraphInfo executionGraphInfo, Throwable throwable) { - return new DispatcherJobResult(executionGraphInfo, throwable); - } - - public static DispatcherJobResult forSuccess(ExecutionGraphInfo executionGraphInfo) { - return new DispatcherJobResult(executionGraphInfo, null); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index 832a88aa7ce91..47883bf47ab56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -54,7 +54,7 @@ public MiniDispatcher createDispatcher( fencingToken, DispatcherServices.from( partialDispatcherServicesWithJobGraphStore, - DefaultJobManagerRunnerFactory.INSTANCE), + JobMasterServiceLeadershipRunnerFactory.INSTANCE), jobGraph, dispatcherBootstrapFactory, executionMode); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java similarity index 63% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java index 10726d709f3d8..1ab28eca7519d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DefaultJobManagerRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -21,26 +21,31 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.SchedulerExecutionMode; +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory; import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobMasterConfiguration; +import org.apache.flink.runtime.jobmaster.JobMasterServiceLeadershipRunner; import org.apache.flink.runtime.jobmaster.SlotPoolServiceSchedulerFactory; import org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory; +import org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceProcessFactory; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; -import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.ShuffleServiceLoader; import org.apache.flink.util.Preconditions; -/** Singleton default factory for {@link JobManagerRunnerImpl}. */ -public enum DefaultJobManagerRunnerFactory implements JobManagerRunnerFactory { +import static org.apache.flink.util.Preconditions.checkArgument; + +/** Factory which creates a {@link JobMasterServiceLeadershipRunner}. */ +public enum JobMasterServiceLeadershipRunnerFactory implements JobManagerRunnerFactory { INSTANCE; @Override @@ -56,9 +61,16 @@ public JobManagerRunner createJobManagerRunner( long initializationTimestamp) throws Exception { + checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty"); + final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); + final RunningJobsRegistry runningJobsRegistry = + highAvailabilityServices.getRunningJobsRegistry(); + final LeaderElectionService jobManagerLeaderElectionService = + highAvailabilityServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); + final SlotPoolServiceSchedulerFactory slotPoolServiceSchedulerFactory = DefaultSlotPoolServiceSchedulerFactory.fromConfiguration( configuration, jobGraph.getJobType()); @@ -75,27 +87,45 @@ public JobManagerRunner createJobManagerRunner( ShuffleServiceLoader.loadShuffleServiceFactory(configuration) .createShuffleMaster(configuration); - final JobMasterServiceFactory jobMasterFactory = + final LibraryCacheManager.ClassLoaderLease classLoaderLease = + jobManagerServices + .getLibraryCacheManager() + .registerClassLoaderLease(jobGraph.getJobID()); + + final ClassLoader userCodeClassLoader = + classLoaderLease + .getOrResolveClassLoader( + jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths()) + .asClassLoader(); + + final DefaultJobMasterServiceFactory jobMasterServiceFactory = new DefaultJobMasterServiceFactory( - jobMasterConfiguration, - slotPoolServiceSchedulerFactory, + jobManagerServices.getScheduledExecutorService(), rpcService, + jobMasterConfiguration, + jobGraph, highAvailabilityServices, + slotPoolServiceSchedulerFactory, jobManagerServices, heartbeatServices, jobManagerJobMetricGroupFactory, fatalErrorHandler, - shuffleMaster); + userCodeClassLoader, + shuffleMaster, + initializationTimestamp); - return new JobManagerRunnerImpl( - jobGraph, - jobMasterFactory, - highAvailabilityServices, - jobManagerServices - .getLibraryCacheManager() - .registerClassLoaderLease(jobGraph.getJobID()), - jobManagerServices.getScheduledExecutorService(), - fatalErrorHandler, - initializationTimestamp); + final DefaultJobMasterServiceProcessFactory jobMasterServiceProcessFactory = + new DefaultJobMasterServiceProcessFactory( + jobGraph.getJobID(), + jobGraph.getName(), + initializationTimestamp, + jobMasterServiceFactory); + + return new JobMasterServiceLeadershipRunner( + jobMasterServiceProcessFactory, + jobManagerLeaderElectionService, + runningJobsRegistry, + classLoaderLease, + fatalErrorHandler); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index 4c113d57bb587..55d92efa366ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -43,6 +43,6 @@ public StandaloneDispatcher createDispatcher( dispatcherBootstrapFactory, DispatcherServices.from( partialDispatcherServicesWithJobGraphStore, - DefaultJobManagerRunnerFactory.INSTANCE)); + JobMasterServiceLeadershipRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcess.java new file mode 100644 index 0000000000000..d417ee2f416c3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcess.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.client.JobInitializationException; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.util.FlinkException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; + +/** + * Default {@link JobMasterServiceProcess} which is responsible for creating and running a {@link + * JobMasterService}. The process is responsible for receiving the signals from the {@link + * JobMasterService} and to create the respective {@link JobManagerRunnerResult} from it. + * + *

The {@link JobMasterService} can be created asynchronously and the creation can also fail. + * That is why the process needs to observe the creation operation and complete the {@link + * #resultFuture} with an initialization failure. + * + *

The {@link #resultFuture} can be completed with the following values: + * + *

    + *
  • {@link JobManagerRunnerResult} to signal an initialization failure of the {@link + * JobMasterService} or the completion of a job + *
  • {@link JobNotFinishedException} to signal that the job has not been completed by the {@link + * JobMasterService} + *
  • {@link Exception} to signal an unexpected failure + *
+ */ +public class DefaultJobMasterServiceProcess + implements JobMasterServiceProcess, OnCompletionActions { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultJobMasterServiceProcess.class); + + private final Object lock = new Object(); + + private final JobID jobId; + + private final UUID leaderSessionId; + + private final CompletableFuture jobMasterServiceFuture; + + private final CompletableFuture terminationFuture = new CompletableFuture<>(); + + private final CompletableFuture resultFuture = + new CompletableFuture<>(); + + private final CompletableFuture jobMasterGatewayFuture = + new CompletableFuture<>(); + + private final CompletableFuture leaderAddressFuture = new CompletableFuture<>(); + + @GuardedBy("lock") + private boolean isRunning = true; + + public DefaultJobMasterServiceProcess( + JobID jobId, + UUID leaderSessionId, + JobMasterServiceFactory jobMasterServiceFactory, + Function failedArchivedExecutionGraphFactory) { + this.jobId = jobId; + this.leaderSessionId = leaderSessionId; + this.jobMasterServiceFuture = + jobMasterServiceFactory.createJobMasterService(leaderSessionId, this); + + jobMasterServiceFuture.whenComplete( + (jobMasterService, throwable) -> { + if (throwable != null) { + final JobInitializationException jobInitializationException = + new JobInitializationException( + jobId, "Could not start the JobMaster.", throwable); + + LOG.debug( + "Initialization of the JobMasterService for job {} under leader id {} failed.", + jobId, + leaderSessionId, + jobInitializationException); + + resultFuture.complete( + JobManagerRunnerResult.forInitializationFailure( + new ExecutionGraphInfo( + failedArchivedExecutionGraphFactory.apply( + jobInitializationException)), + jobInitializationException)); + } else { + registerJobMasterServiceFutures(jobMasterService); + } + }); + } + + private void registerJobMasterServiceFutures(JobMasterService jobMasterService) { + LOG.debug( + "Successfully created the JobMasterService for job {} under leader id {}.", + jobId, + leaderSessionId); + jobMasterGatewayFuture.complete(jobMasterService.getGateway()); + leaderAddressFuture.complete(jobMasterService.getAddress()); + + jobMasterService + .getTerminationFuture() + .whenComplete( + (unused, throwable) -> { + synchronized (lock) { + if (isRunning) { + LOG.warn( + "Unexpected termination of the JobMasterService for job {} under leader id {}.", + jobId, + leaderSessionId); + jobMasterFailed( + new FlinkException( + "Unexpected termination of the JobMasterService.", + throwable)); + } + } + }); + } + + @Override + public CompletableFuture closeAsync() { + synchronized (lock) { + if (isRunning) { + isRunning = false; + + LOG.debug( + "Terminating the JobMasterService process for job {} under leader id {}.", + jobId, + leaderSessionId); + + resultFuture.completeExceptionally(new JobNotFinishedException(jobId)); + jobMasterGatewayFuture.completeExceptionally( + new FlinkException("Process has been closed.")); + + jobMasterServiceFuture.whenComplete( + (jobMasterService, throwable) -> { + if (throwable != null) { + // JobMasterService creation has failed. Nothing to stop then :-) + terminationFuture.complete(null); + } else { + FutureUtils.forward( + jobMasterService.closeAsync(), terminationFuture); + } + }); + + terminationFuture.whenComplete( + (unused, throwable) -> + LOG.debug( + "JobMasterService process for job {} under leader id {} has been terminated.", + jobId, + leaderSessionId)); + } + } + return terminationFuture; + } + + @Override + public boolean isInitializedAndRunning() { + synchronized (lock) { + return jobMasterServiceFuture.isDone() + && !jobMasterServiceFuture.isCompletedExceptionally() + && isRunning; + } + } + + @Override + public CompletableFuture getJobMasterGatewayFuture() { + return jobMasterGatewayFuture; + } + + @Override + public CompletableFuture getResultFuture() { + return resultFuture; + } + + @Override + public CompletableFuture getLeaderAddressFuture() { + return leaderAddressFuture; + } + + @Override + public void jobReachedGloballyTerminalState(ExecutionGraphInfo executionGraphInfo) { + LOG.debug( + "Job {} under leader id {} reached a globally terminal state {}.", + jobId, + leaderSessionId, + executionGraphInfo.getArchivedExecutionGraph().getState()); + resultFuture.complete(JobManagerRunnerResult.forSuccess(executionGraphInfo)); + } + + @Override + public void jobFinishedByOther() { + LOG.debug( + "Job {} under leader id {} has been finished by another process.", + jobId, + leaderSessionId); + resultFuture.completeExceptionally(new JobNotFinishedException(jobId)); + } + + @Override + public void jobMasterFailed(Throwable cause) { + LOG.debug("Job {} under leader id {} failed.", jobId, leaderSessionId); + resultFuture.completeExceptionally(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index fbd40a384191f..cc8373f3fa5fb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -19,6 +19,11 @@ package org.apache.flink.runtime.jobmaster; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.util.AutoCloseableAsync; import java.util.concurrent.CompletableFuture; @@ -43,7 +48,10 @@ public interface JobManagerRunner extends AutoCloseableAsync { /** * Get the result future of this runner. The future is completed once the executed job reaches a - * globally terminal state. + * globally terminal state or if the initialization of the {@link JobMaster} fails. If the + * result future is completed exceptionally via {@link JobNotFinishedException}, then this + * signals that the job has not been completed successfully. All other exceptional completions + * denote an unexpected exception which leads to a process restart. * * @return Future which is completed with the job result */ @@ -55,4 +63,43 @@ public interface JobManagerRunner extends AutoCloseableAsync { * @return job id of the executed job */ JobID getJobID(); + + /** + * Cancels the currently executed job. + * + * @param timeout of this operation + * @return Future acknowledge of the operation + */ + CompletableFuture cancel(Time timeout); + + /** + * Requests the current job status. + * + * @param timeout for the rpc call + * @return Future containing the current job status + */ + CompletableFuture requestJobStatus(Time timeout); + + /** + * Request the details of the executed job. + * + * @param timeout for the rpc call + * @return Future details of the executed job + */ + CompletableFuture requestJobDetails(Time timeout); + + /** + * Requests the {@link ExecutionGraphInfo} of the executed job. + * + * @param timeout for the rpc call + * @return Future which is completed with the {@link ExecutionGraphInfo} of the executed job + */ + CompletableFuture requestJob(Time timeout); + + /** + * Flag indicating if the JobManagerRunner has been initialized. + * + * @return true if the JobManagerRunner has been initialized. + */ + boolean isInitialized(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java deleted file mode 100644 index 042945d9249cf..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImpl.java +++ /dev/null @@ -1,504 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.jobmaster; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.client.JobInitializationException; -import org.apache.flink.runtime.concurrent.FutureUtils; -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; -import org.apache.flink.runtime.leaderelection.LeaderContender; -import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.function.ThrowingRunnable; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; - -import java.io.IOException; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; - -import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.flink.util.Preconditions.checkState; - -/** - * The runner for the job manager. It deals with job level leader election and make underlying job - * manager properly reacted. - */ -public class JobManagerRunnerImpl - implements LeaderContender, OnCompletionActions, JobManagerRunner { - - private static final Logger log = LoggerFactory.getLogger(JobManagerRunnerImpl.class); - - // ------------------------------------------------------------------------ - - /** - * Lock to ensure that this runner can deal with leader election event and job completion - * notifies simultaneously. - */ - private final Object lock = new Object(); - - /** The job graph needs to run. */ - private final JobGraph jobGraph; - - /** Used to check whether a job needs to be run. */ - private final RunningJobsRegistry runningJobsRegistry; - - /** Leader election for this job. */ - private final LeaderElectionService leaderElectionService; - - private final LibraryCacheManager.ClassLoaderLease classLoaderLease; - - private final Executor executor; - - private final FatalErrorHandler fatalErrorHandler; - - private final CompletableFuture resultFuture; - - private final CompletableFuture terminationFuture; - - private final JobMasterServiceFactory jobMasterServiceFactory; - - private final ClassLoader userCodeClassLoader; - - private final long initializationTimestamp; - - private CompletableFuture leadershipOperation; - - @Nullable private JobMasterService jobMasterService; - - /** flag marking the runner as shut down. */ - private volatile boolean shutdown; - - private volatile CompletableFuture leaderGatewayFuture; - - // ------------------------------------------------------------------------ - - /** - * Exceptions that occur while creating the JobManager or JobManagerRunnerImpl are directly - * thrown and not reported to the given {@code FatalErrorHandler}. - * - * @throws Exception Thrown if the runner cannot be set up, because either one of the required - * services could not be started, or the Job could not be initialized. - */ - public JobManagerRunnerImpl( - final JobGraph jobGraph, - final JobMasterServiceFactory jobMasterServiceFactory, - final HighAvailabilityServices haServices, - final LibraryCacheManager.ClassLoaderLease classLoaderLease, - final Executor executor, - final FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) - throws Exception { - - this.resultFuture = new CompletableFuture<>(); - this.terminationFuture = new CompletableFuture<>(); - this.leadershipOperation = CompletableFuture.completedFuture(null); - - this.jobGraph = checkNotNull(jobGraph); - this.jobMasterServiceFactory = checkNotNull(jobMasterServiceFactory); - this.classLoaderLease = checkNotNull(classLoaderLease); - this.executor = checkNotNull(executor); - this.fatalErrorHandler = checkNotNull(fatalErrorHandler); - this.initializationTimestamp = initializationTimestamp; - - checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty"); - - // libraries and class loader first - try { - userCodeClassLoader = - classLoaderLease - .getOrResolveClassLoader( - jobGraph.getUserJarBlobKeys(), jobGraph.getClasspaths()) - .asClassLoader(); - } catch (IOException e) { - throw new Exception("Cannot set up the user code libraries: " + e.getMessage(), e); - } - - // high availability services next - this.runningJobsRegistry = haServices.getRunningJobsRegistry(); - this.leaderElectionService = - haServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); - - this.leaderGatewayFuture = new CompletableFuture<>(); - } - - // ---------------------------------------------------------------------------------------------- - // Getter - // ---------------------------------------------------------------------------------------------- - - @Override - public CompletableFuture getJobMasterGateway() { - return leaderGatewayFuture; - } - - @Override - public CompletableFuture getResultFuture() { - return resultFuture; - } - - @Override - public JobID getJobID() { - return jobGraph.getJobID(); - } - - // ---------------------------------------------------------------------------------------------- - // Lifecycle management - // ---------------------------------------------------------------------------------------------- - - @Override - public void start() throws Exception { - try { - leaderElectionService.start(this); - } catch (Exception e) { - log.error( - "Could not start the JobManager because the leader election service did not start.", - e); - throw new Exception("Could not start the leader election service.", e); - } - } - - @Override - public CompletableFuture closeAsync() { - synchronized (lock) { - if (!shutdown) { - shutdown = true; - - setNewLeaderGatewayFuture(); - leaderGatewayFuture.completeExceptionally( - new FlinkException("JobMaster has been shut down.")); - - final CompletableFuture jobManagerTerminationFuture; - - if (jobMasterService == null) { - jobManagerTerminationFuture = FutureUtils.completedVoidFuture(); - } else { - jobManagerTerminationFuture = jobMasterService.closeAsync(); - } - - jobManagerTerminationFuture.whenComplete( - (Void ignored, Throwable throwable) -> { - try { - leaderElectionService.stop(); - } catch (Throwable t) { - throwable = - ExceptionUtils.firstOrSuppressed( - t, - ExceptionUtils.stripCompletionException(throwable)); - } - - classLoaderLease.release(); - - resultFuture.complete(JobManagerRunnerResult.forJobNotFinished()); - - if (throwable != null) { - terminationFuture.completeExceptionally( - new FlinkException( - "Could not properly shut down the JobManagerRunner", - throwable)); - } else { - terminationFuture.complete(null); - } - }); - } - - return terminationFuture; - } - } - - // ---------------------------------------------------------------------------------------------- - // Result and error handling methods - // ---------------------------------------------------------------------------------------------- - - /** Job completion notification triggered by JobManager. */ - @Override - public void jobReachedGloballyTerminalState(ExecutionGraphInfo executionGraphInfo) { - unregisterJobFromHighAvailability(); - // complete the result future with the information of the information of the terminated job - resultFuture.complete(JobManagerRunnerResult.forSuccess(executionGraphInfo)); - } - - /** Job completion notification triggered by self. */ - @Override - public void jobFinishedByOther() { - resultFuture.complete(JobManagerRunnerResult.forJobNotFinished()); - } - - @Override - public void jobMasterFailed(Throwable cause) { - handleJobManagerRunnerError(cause); - } - - private void handleJobManagerRunnerError(Throwable cause) { - if (ExceptionUtils.isJvmFatalError(cause)) { - fatalErrorHandler.onFatalError(cause); - } else { - resultFuture.completeExceptionally(cause); - } - } - - /** - * Marks this runner's job as not running. Other JobManager will not recover the job after this - * call. - * - *

This method never throws an exception. - */ - private void unregisterJobFromHighAvailability() { - try { - runningJobsRegistry.setJobFinished(jobGraph.getJobID()); - } catch (Throwable t) { - log.error( - "Could not un-register from high-availability services job {} ({})." - + "Other JobManager's may attempt to recover it and re-execute it.", - jobGraph.getName(), - jobGraph.getJobID(), - t); - } - } - - // ---------------------------------------------------------------------------------------------- - // Leadership methods - // ---------------------------------------------------------------------------------------------- - - @Override - public void grantLeadership(final UUID leaderSessionID) { - synchronized (lock) { - if (shutdown) { - log.debug( - "JobManagerRunner cannot be granted leadership because it is already shut down."); - return; - } - - leadershipOperation = - leadershipOperation.thenRun( - ThrowingRunnable.unchecked( - () -> { - synchronized (lock) { - verifyJobSchedulingStatusAndStartJobManager( - leaderSessionID); - } - })); - - handleException(leadershipOperation, "Could not start the job manager."); - } - } - - @GuardedBy("lock") - private void verifyJobSchedulingStatusAndStartJobManager(UUID leaderSessionId) - throws FlinkException { - if (shutdown) { - log.debug("Ignoring starting JobMaster because JobManagerRunner is already shut down."); - return; - } - - final RunningJobsRegistry.JobSchedulingStatus jobSchedulingStatus = - getJobSchedulingStatus(); - - if (jobSchedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE) { - jobAlreadyDone(); - } else { - startJobMaster(leaderSessionId); - } - } - - @GuardedBy("lock") - private void startJobMaster(UUID leaderSessionId) throws FlinkException { - log.info( - "JobManager runner for job {} ({}) was granted leadership with session id {}.", - jobGraph.getName(), - jobGraph.getJobID(), - leaderSessionId); - - try { - runningJobsRegistry.setJobRunning(jobGraph.getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Failed to set the job %s to running in the running jobs registry.", - jobGraph.getJobID()), - e); - } - - startJobMasterServiceSafely(leaderSessionId); - - if (jobMasterService != null) { - confirmLeaderSessionIdIfStillLeader(jobMasterService, leaderSessionId); - } - } - - private void startJobMasterServiceSafely(UUID leaderSessionId) { - checkState(jobMasterService == null, "JobMasterService must be null before being started."); - - try { - final JobMasterService newJobMasterService = - jobMasterServiceFactory.createJobMasterService( - jobGraph, - new JobMasterId(leaderSessionId), - this, - userCodeClassLoader, - initializationTimestamp); - - jobMasterService = newJobMasterService; - - jobMasterService - .getTerminationFuture() - .whenComplete( - (unused, throwable) -> { - if (throwable != null) { - synchronized (lock) { - // check that we are still running and the JobMasterService - // is still valid - if (!shutdown && newJobMasterService == jobMasterService) { - handleJobManagerRunnerError(throwable); - } - } - } - }); - } catch (Exception e) { - resultFuture.complete( - JobManagerRunnerResult.forInitializationFailure( - new JobInitializationException( - jobGraph.getJobID(), "Could not start the JobMaster.", e))); - } - } - - private void jobAlreadyDone() { - log.info("Granted leader ship but job {} has been finished. ", jobGraph.getJobID()); - jobFinishedByOther(); - } - - private RunningJobsRegistry.JobSchedulingStatus getJobSchedulingStatus() throws FlinkException { - try { - return runningJobsRegistry.getJobSchedulingStatus(jobGraph.getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Could not retrieve the job scheduling status for job %s.", - jobGraph.getJobID()), - e); - } - } - - private void confirmLeaderSessionIdIfStillLeader( - JobMasterService jobMasterService, UUID leaderSessionId) { - - if (leaderElectionService.hasLeadership(leaderSessionId)) { - leaderGatewayFuture.complete(jobMasterService.getGateway()); - leaderElectionService.confirmLeadership(leaderSessionId, jobMasterService.getAddress()); - } else { - log.debug( - "Ignoring confirmation of leader session id because {} is no longer the leader.", - getDescription()); - } - } - - @Override - public void revokeLeadership() { - synchronized (lock) { - if (shutdown) { - log.debug( - "Ignoring revoking leadership because JobManagerRunner is already shut down."); - return; - } - - leadershipOperation = - leadershipOperation.thenCompose( - (ignored) -> { - synchronized (lock) { - return revokeJobMasterLeadership(); - } - }); - - handleException(leadershipOperation, "Could not suspend the job manager."); - } - } - - @GuardedBy("lock") - private CompletableFuture revokeJobMasterLeadership() { - if (shutdown) { - log.debug( - "Ignoring revoking JobMaster leadership because JobManagerRunner is already shut down."); - return FutureUtils.completedVoidFuture(); - } - - if (jobMasterService != null) { - log.info( - "JobManager for job {} ({}) at {} was revoked leadership.", - jobGraph.getName(), - jobGraph.getJobID(), - jobMasterService.getAddress()); - - setNewLeaderGatewayFuture(); - - final CompletableFuture jobMasterServiceTerminationFuture = - jobMasterService.closeAsync(); - jobMasterService = null; - - return jobMasterServiceTerminationFuture; - } else { - return FutureUtils.completedVoidFuture(); - } - } - - private void handleException(CompletableFuture leadershipOperation, String message) { - leadershipOperation.whenComplete( - (ignored, throwable) -> { - if (throwable != null) { - handleJobManagerRunnerError(new FlinkException(message, throwable)); - } - }); - } - - @GuardedBy("lock") - private void setNewLeaderGatewayFuture() { - final CompletableFuture oldLeaderGatewayFuture = leaderGatewayFuture; - - leaderGatewayFuture = new CompletableFuture<>(); - - if (!oldLeaderGatewayFuture.isDone()) { - leaderGatewayFuture.whenComplete( - (JobMasterGateway jobMasterGateway, Throwable throwable) -> { - if (throwable != null) { - oldLeaderGatewayFuture.completeExceptionally(throwable); - } else { - oldLeaderGatewayFuture.complete(jobMasterGateway); - } - }); - } - } - - @Override - public void handleError(Exception exception) { - log.error("Leader Election Service encountered a fatal error.", exception); - handleJobManagerRunnerError(exception); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java index 53542ef4d3a0d..e8102cb1471c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResult.java @@ -28,36 +28,25 @@ /** The result of the {@link JobManagerRunner}. */ public final class JobManagerRunnerResult { - @Nullable private final ExecutionGraphInfo executionGraphInfo; + private final ExecutionGraphInfo executionGraphInfo; @Nullable private final Throwable failure; private JobManagerRunnerResult( - @Nullable ExecutionGraphInfo executionGraphInfo, @Nullable Throwable failure) { + ExecutionGraphInfo executionGraphInfo, @Nullable Throwable failure) { this.executionGraphInfo = executionGraphInfo; this.failure = failure; } public boolean isSuccess() { - return executionGraphInfo != null && failure == null; - } - - public boolean isJobNotFinished() { - return executionGraphInfo == null && failure == null; + return failure == null; } public boolean isInitializationFailure() { - return executionGraphInfo == null && failure != null; + return failure != null; } - /** - * This method returns the payload of the successful JobManagerRunnerResult. - * - * @return the @link ExecutionGraphInfo} of a successfully finished job - * @throws IllegalStateException if the result is not a success - */ public ExecutionGraphInfo getExecutionGraphInfo() { - Preconditions.checkState(isSuccess()); return executionGraphInfo; } @@ -90,15 +79,12 @@ public int hashCode() { return Objects.hash(executionGraphInfo, failure); } - public static JobManagerRunnerResult forJobNotFinished() { - return new JobManagerRunnerResult(null, null); - } - public static JobManagerRunnerResult forSuccess(ExecutionGraphInfo executionGraphInfo) { return new JobManagerRunnerResult(executionGraphInfo, null); } - public static JobManagerRunnerResult forInitializationFailure(Throwable failure) { - return new JobManagerRunnerResult(null, failure); + public static JobManagerRunnerResult forInitializationFailure( + ExecutionGraphInfo executionGraphInfo, Throwable failure) { + return new JobManagerRunnerResult(executionGraphInfo, failure); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java new file mode 100644 index 0000000000000..a3ca212994310 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java @@ -0,0 +1,537 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; +import org.apache.flink.runtime.leaderelection.LeaderContender; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.io.IOException; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.Supplier; + +/** + * Leadership runner for the {@link JobMasterServiceProcess}. + * + *

The responsibility of this component is to manage the leadership of the {@link + * JobMasterServiceProcess}. This means that the runner will create an instance of the process when + * it obtains the leadership. The process is stopped once the leadership is revoked. + * + *

This component only accepts signals (job result completion, initialization failure) as long as + * it is running and as long as the signals are coming from the current leader process. This ensures + * that only the current leader can affect this component. + * + *

All leadership operations are serialized. This means that granting the leadership has to + * complete before the leadership can be revoked and vice versa. + * + *

The {@link #resultFuture} can be completed with the following values: * * + * + *

    + *
  • {@link JobManagerRunnerResult} to signal an initialization failure of the {@link + * JobMasterService} or the completion of a job + *
  • {@link JobNotFinishedException} to signal that the job has not been completed by the {@link + * JobMasterService} + *
  • {@link Exception} to signal an unexpected failure + *
+ */ +public class JobMasterServiceLeadershipRunner implements JobManagerRunner, LeaderContender { + + private static final Logger LOG = + LoggerFactory.getLogger(JobMasterServiceLeadershipRunner.class); + + private final Object lock = new Object(); + + private final JobMasterServiceProcessFactory jobMasterServiceProcessFactory; + + private final LeaderElectionService leaderElectionService; + + private final RunningJobsRegistry runningJobsRegistry; + + private final LibraryCacheManager.ClassLoaderLease classLoaderLease; + + private final FatalErrorHandler fatalErrorHandler; + + private final CompletableFuture terminationFuture = new CompletableFuture<>(); + + private final CompletableFuture resultFuture = + new CompletableFuture<>(); + + @GuardedBy("lock") + private State state = State.RUNNING; + + @GuardedBy("lock") + private CompletableFuture sequentialOperation = FutureUtils.completedVoidFuture(); + + @GuardedBy("lock") + private JobMasterServiceProcess jobMasterServiceProcess = + JobMasterServiceProcess.waitingForLeadership(); + + @GuardedBy("lock") + private CompletableFuture jobMasterGatewayFuture = new CompletableFuture<>(); + + @GuardedBy("lock") + private boolean hasCurrentLeaderBeenCancelled = false; + + public JobMasterServiceLeadershipRunner( + JobMasterServiceProcessFactory jobMasterServiceProcessFactory, + LeaderElectionService leaderElectionService, + RunningJobsRegistry runningJobsRegistry, + LibraryCacheManager.ClassLoaderLease classLoaderLease, + FatalErrorHandler fatalErrorHandler) { + this.jobMasterServiceProcessFactory = jobMasterServiceProcessFactory; + this.leaderElectionService = leaderElectionService; + this.runningJobsRegistry = runningJobsRegistry; + this.classLoaderLease = classLoaderLease; + this.fatalErrorHandler = fatalErrorHandler; + } + + @Override + public CompletableFuture closeAsync() { + synchronized (lock) { + if (state != State.STOPPED) { + state = State.STOPPED; + + LOG.debug("Terminating the leadership runner for job {}.", getJobID()); + + jobMasterGatewayFuture.completeExceptionally( + new FlinkException( + "JobMasterServiceLeadershipRunner is closed. Therefore, the corresponding JobMaster will never acquire the leadership.")); + resultFuture.completeExceptionally(new JobNotFinishedException(getJobID())); + + final CompletableFuture processTerminationFuture = + jobMasterServiceProcess.closeAsync(); + + final CompletableFuture serviceTerminationFuture = + FutureUtils.runAfterwards( + processTerminationFuture, + () -> { + classLoaderLease.release(); + leaderElectionService.stop(); + }); + + FutureUtils.forward(serviceTerminationFuture, terminationFuture); + + terminationFuture.whenComplete( + (unused, throwable) -> + LOG.debug( + "Leadership runner for job {} has been terminated.", + getJobID())); + } + } + + return terminationFuture; + } + + @Override + public void start() throws Exception { + LOG.debug("Start leadership runner for job {}.", getJobID()); + leaderElectionService.start(this); + } + + @Override + public CompletableFuture getJobMasterGateway() { + synchronized (lock) { + return jobMasterGatewayFuture; + } + } + + @Override + public CompletableFuture getResultFuture() { + return resultFuture; + } + + @Override + public JobID getJobID() { + return jobMasterServiceProcessFactory.getJobId(); + } + + @Override + public CompletableFuture cancel(Time timeout) { + synchronized (lock) { + hasCurrentLeaderBeenCancelled = true; + return getJobMasterGateway() + .thenCompose(jobMasterGateway -> jobMasterGateway.cancel(timeout)) + .exceptionally( + e -> { + throw new CompletionException( + new FlinkException( + "Cancellation failed.", + ExceptionUtils.stripCompletionException(e))); + }); + } + } + + @Override + public CompletableFuture requestJobStatus(Time timeout) { + return requestJob(timeout) + .thenApply( + executionGraphInfo -> + executionGraphInfo.getArchivedExecutionGraph().getState()); + } + + @Override + public CompletableFuture requestJobDetails(Time timeout) { + return requestJob(timeout) + .thenApply( + executionGraphInfo -> + JobDetails.createDetailsForJob( + executionGraphInfo.getArchivedExecutionGraph())); + } + + @Override + public CompletableFuture requestJob(Time timeout) { + synchronized (lock) { + if (state == State.RUNNING) { + if (jobMasterServiceProcess.isInitializedAndRunning()) { + return getJobMasterGateway() + .thenCompose(jobMasterGateway -> jobMasterGateway.requestJob(timeout)); + } else { + return CompletableFuture.completedFuture( + new ExecutionGraphInfo( + jobMasterServiceProcessFactory.createArchivedExecutionGraph( + hasCurrentLeaderBeenCancelled + ? JobStatus.CANCELLING + : JobStatus.INITIALIZING, + null))); + } + } else { + return resultFuture.thenApply(JobManagerRunnerResult::getExecutionGraphInfo); + } + } + } + + @Override + public boolean isInitialized() { + synchronized (lock) { + return jobMasterServiceProcess.isInitializedAndRunning(); + } + } + + @Override + public void grantLeadership(UUID leaderSessionID) { + runIfStateRunning( + () -> startJobMasterServiceProcessAsync(leaderSessionID), + "starting a new JobMasterServiceProcess"); + } + + @GuardedBy("lock") + private void startJobMasterServiceProcessAsync(UUID leaderSessionId) { + sequentialOperation = + sequentialOperation.thenRun( + () -> + runIfValidLeader( + leaderSessionId, + ThrowingRunnable.unchecked( + () -> + verifyJobSchedulingStatusAndCreateJobMasterServiceProcess( + leaderSessionId)), + "verify job scheduling status and create JobMasterServiceProcess")); + + handleAsyncOperationError(sequentialOperation, "Could not start the job manager."); + } + + @GuardedBy("lock") + private void verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId) + throws FlinkException { + final RunningJobsRegistry.JobSchedulingStatus jobSchedulingStatus = + getJobSchedulingStatus(); + + if (jobSchedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE) { + jobAlreadyDone(); + } else { + createNewJobMasterServiceProcess(leaderSessionId); + } + } + + private void jobAlreadyDone() { + resultFuture.completeExceptionally(new JobNotFinishedException(getJobID())); + } + + private RunningJobsRegistry.JobSchedulingStatus getJobSchedulingStatus() throws FlinkException { + try { + return runningJobsRegistry.getJobSchedulingStatus(getJobID()); + } catch (IOException e) { + throw new FlinkException( + String.format( + "Could not retrieve the job scheduling status for job %s.", getJobID()), + e); + } + } + + @GuardedBy("lock") + private void createNewJobMasterServiceProcess(UUID leaderSessionId) throws FlinkException { + Preconditions.checkState(jobMasterServiceProcess.closeAsync().isDone()); + + LOG.debug( + "Create new JobMasterServiceProcess because we were granted leadership under {}.", + leaderSessionId); + + try { + runningJobsRegistry.setJobRunning(getJobID()); + } catch (IOException e) { + throw new FlinkException( + String.format( + "Failed to set the job %s to running in the running jobs registry.", + getJobID()), + e); + } + + jobMasterServiceProcess = jobMasterServiceProcessFactory.create(leaderSessionId); + + forwardIfValidLeader( + leaderSessionId, + jobMasterServiceProcess.getJobMasterGatewayFuture(), + jobMasterGatewayFuture, + "JobMasterGatewayFuture from JobMasterServiceProcess"); + forwardResultFuture(leaderSessionId, jobMasterServiceProcess.getResultFuture()); + confirmLeadership(leaderSessionId, jobMasterServiceProcess.getLeaderAddressFuture()); + } + + private void confirmLeadership( + UUID leaderSessionId, CompletableFuture leaderAddressFuture) { + FutureUtils.assertNoException( + leaderAddressFuture.thenAccept( + address -> { + synchronized (lock) { + if (isValidLeader(leaderSessionId)) { + LOG.debug("Confirm leadership {}.", leaderSessionId); + leaderElectionService.confirmLeadership( + leaderSessionId, address); + } else { + LOG.trace( + "Ignore confirming leadership because the leader {} is no longer valid.", + leaderSessionId); + } + } + })); + } + + private void forwardResultFuture( + UUID leaderSessionId, CompletableFuture resultFuture) { + resultFuture.whenComplete( + (jobManagerRunnerResult, throwable) -> { + synchronized (lock) { + if (isValidLeader(leaderSessionId)) { + onJobCompletion(jobManagerRunnerResult, throwable); + } else { + LOG.trace( + "Ignore result future forwarding because the leader {} is no longer valid.", + leaderSessionId); + } + } + }); + } + + @GuardedBy("lock") + private void onJobCompletion( + JobManagerRunnerResult jobManagerRunnerResult, Throwable throwable) { + state = State.JOB_COMPLETED; + + LOG.debug("Completing the result for job {}.", getJobID()); + + if (throwable != null) { + resultFuture.completeExceptionally(throwable); + jobMasterGatewayFuture.completeExceptionally( + new FlinkException( + "Could not retrieve JobMasterGateway because the JobMaster failed.", + throwable)); + } else { + if (jobManagerRunnerResult.isSuccess()) { + try { + runningJobsRegistry.setJobFinished(getJobID()); + } catch (IOException e) { + LOG.error( + "Could not un-register from high-availability services job {}." + + "Other JobManager's may attempt to recover it and re-execute it.", + getJobID(), + e); + } + } else { + jobMasterGatewayFuture.completeExceptionally( + new FlinkException( + "Could not retrieve JobMasterGateway because the JobMaster initialization failed.", + jobManagerRunnerResult.getInitializationFailure())); + } + + resultFuture.complete(jobManagerRunnerResult); + } + } + + @Override + public void revokeLeadership() { + runIfStateRunning( + this::stopJobMasterServiceProcessAsync, + "revoke leadership from JobMasterServiceProcess"); + } + + @GuardedBy("lock") + private void stopJobMasterServiceProcessAsync() { + sequentialOperation = + sequentialOperation.thenCompose( + ignored -> + callIfRunning( + this::stopJobMasterServiceProcess, + "stop leading JobMasterServiceProcess") + .orElse(FutureUtils.completedVoidFuture())); + + handleAsyncOperationError(sequentialOperation, "Could not suspend the job manager."); + } + + @GuardedBy("lock") + private CompletableFuture stopJobMasterServiceProcess() { + LOG.debug("Stop current JobMasterServiceProcess because the leadership has been revoked."); + + jobMasterGatewayFuture.completeExceptionally( + new FlinkException( + "Cannot obtain JobMasterGateway because the JobMaster lost leadership.")); + jobMasterGatewayFuture = new CompletableFuture<>(); + + hasCurrentLeaderBeenCancelled = false; + + return jobMasterServiceProcess.closeAsync(); + } + + @Override + public void handleError(Exception exception) { + fatalErrorHandler.onFatalError(exception); + } + + private void handleAsyncOperationError(CompletableFuture operation, String message) { + operation.whenComplete( + (unused, throwable) -> { + if (throwable != null) { + runIfStateRunning( + () -> + handleJobMasterServiceLeadershipRunnerError( + new FlinkException(message, throwable)), + "handle JobMasterServiceLeadershipRunner error"); + } + }); + } + + private void handleJobMasterServiceLeadershipRunnerError(Throwable cause) { + if (ExceptionUtils.isJvmFatalError(cause)) { + fatalErrorHandler.onFatalError(cause); + } else { + resultFuture.completeExceptionally(cause); + } + } + + private void runIfStateRunning(Runnable action, String actionDescription) { + synchronized (lock) { + if (isRunning()) { + action.run(); + } else { + LOG.trace( + "Ignore '{}' because the leadership runner is no longer running.", + actionDescription); + } + } + } + + private Optional callIfRunning( + Supplier supplier, String supplierDescription) { + synchronized (lock) { + if (isRunning()) { + return Optional.of(supplier.get()); + } else { + LOG.trace( + "Ignore '{}' because the leadership runner is no longer running.", + supplierDescription); + return Optional.empty(); + } + } + } + + @GuardedBy("lock") + private boolean isRunning() { + return state == State.RUNNING; + } + + private void runIfValidLeader( + UUID expectedLeaderId, Runnable action, String actionDescription) { + synchronized (lock) { + if (isValidLeader(expectedLeaderId)) { + action.run(); + } else { + LOG.trace( + "Ignore leader action '{}' because the leadership runner is no longer the valid leader for {}.", + actionDescription, + expectedLeaderId); + } + } + } + + @GuardedBy("lock") + private boolean isValidLeader(UUID expectedLeaderId) { + return isRunning() && leaderElectionService.hasLeadership(expectedLeaderId); + } + + private void forwardIfValidLeader( + UUID expectedLeaderId, + CompletableFuture source, + CompletableFuture target, + String forwardDescription) { + source.whenComplete( + (t, throwable) -> { + synchronized (lock) { + if (isValidLeader(expectedLeaderId)) { + if (throwable != null) { + target.completeExceptionally(throwable); + } else { + target.complete(t); + } + } else { + LOG.trace( + "Ignore forwarding '{}' because the leadership runner is no longer the valid leader for {}.", + forwardDescription, + expectedLeaderId); + } + } + }); + } + + enum State { + RUNNING, + STOPPED, + JOB_COMPLETED, + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceProcess.java new file mode 100644 index 0000000000000..b7df9314bb711 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceProcess.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.util.AutoCloseableAsync; +import org.apache.flink.util.FlinkException; + +import javax.annotation.Nonnull; + +import java.util.concurrent.CompletableFuture; + +/** JobMasterServiceProcess is responsible for running a {@link JobMasterService}. */ +public interface JobMasterServiceProcess extends AutoCloseableAsync { + + static JobMasterServiceProcess waitingForLeadership() { + return WaitingForLeadership.INSTANCE; + } + + /** True iff the {@link JobMasterService} has been initialized and is running. */ + boolean isInitializedAndRunning(); + + /** + * Future which is completed with the {@link JobMasterGateway} once the {@link JobMasterService} + * has been created. Upon closing of the process, this future is completed exceptionally if it + * is still uncompleted. + */ + CompletableFuture getJobMasterGatewayFuture(); + + /** + * Future which is completed with the result of job execution. The job's result can be the + * {@link JobManagerRunnerResult}, {@link JobNotFinishedException} if the job was not finished + * or an {@link Throwable} if an unexpected failure occurs. Upon closing of the process, this + * future is completed exceptionally with {@link JobNotFinishedException}. + */ + CompletableFuture getResultFuture(); + + /** Future which is completed with the {@link JobMasterService} address once it is created. */ + CompletableFuture getLeaderAddressFuture(); + + enum WaitingForLeadership implements JobMasterServiceProcess { + INSTANCE; + + @Override + public CompletableFuture closeAsync() { + return FutureUtils.completedVoidFuture(); + } + + @Override + public boolean isInitializedAndRunning() { + return false; + } + + @Override + public CompletableFuture getJobMasterGatewayFuture() { + return failedOperationFuture(); + } + + @Override + public CompletableFuture getResultFuture() { + return failedOperationFuture(); + } + + @Override + public CompletableFuture getLeaderAddressFuture() { + return failedOperationFuture(); + } + + @Nonnull + private CompletableFuture failedOperationFuture() { + return FutureUtils.completedExceptionally( + new FlinkException("Still waiting for the leadership.")); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java index 6b4335c6ee8ab..637bdc584976e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceFactory.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -30,66 +30,79 @@ import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterConfiguration; import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.JobMasterService; import org.apache.flink.runtime.jobmaster.SlotPoolServiceSchedulerFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.shuffle.ShuffleMaster; +import org.apache.flink.util.function.FunctionUtils; -/** Default implementation of the {@link JobMasterServiceFactory}. */ -public class DefaultJobMasterServiceFactory implements JobMasterServiceFactory { - - private final JobMasterConfiguration jobMasterConfiguration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; - private final SlotPoolServiceSchedulerFactory slotPoolServiceSchedulerFactory; +public class DefaultJobMasterServiceFactory implements JobMasterServiceFactory { + private final Executor executor; private final RpcService rpcService; - + private final JobMasterConfiguration jobMasterConfiguration; + private final JobGraph jobGraph; private final HighAvailabilityServices haServices; - + private final SlotPoolServiceSchedulerFactory slotPoolServiceSchedulerFactory; private final JobManagerSharedServices jobManagerSharedServices; - private final HeartbeatServices heartbeatServices; - private final JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory; - private final FatalErrorHandler fatalErrorHandler; - + private final ClassLoader userCodeClassloader; private final ShuffleMaster shuffleMaster; + private final long initializationTimestamp; public DefaultJobMasterServiceFactory( - JobMasterConfiguration jobMasterConfiguration, - SlotPoolServiceSchedulerFactory slotPoolServiceSchedulerFactory, + Executor executor, RpcService rpcService, + JobMasterConfiguration jobMasterConfiguration, + JobGraph jobGraph, HighAvailabilityServices haServices, + SlotPoolServiceSchedulerFactory slotPoolServiceSchedulerFactory, JobManagerSharedServices jobManagerSharedServices, HeartbeatServices heartbeatServices, JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, FatalErrorHandler fatalErrorHandler, - ShuffleMaster shuffleMaster) { - this.jobMasterConfiguration = jobMasterConfiguration; - this.slotPoolServiceSchedulerFactory = slotPoolServiceSchedulerFactory; + ClassLoader userCodeClassloader, + ShuffleMaster shuffleMaster, + long initializationTimestamp) { + this.executor = executor; this.rpcService = rpcService; + this.jobMasterConfiguration = jobMasterConfiguration; + this.jobGraph = jobGraph; this.haServices = haServices; + this.slotPoolServiceSchedulerFactory = slotPoolServiceSchedulerFactory; this.jobManagerSharedServices = jobManagerSharedServices; this.heartbeatServices = heartbeatServices; this.jobManagerJobMetricGroupFactory = jobManagerJobMetricGroupFactory; this.fatalErrorHandler = fatalErrorHandler; + this.userCodeClassloader = userCodeClassloader; this.shuffleMaster = shuffleMaster; + this.initializationTimestamp = initializationTimestamp; } @Override - public JobMaster createJobMasterService( - JobGraph jobGraph, - JobMasterId jobMasterId, - OnCompletionActions jobCompletionActions, - ClassLoader userCodeClassloader, - long initializationTimestamp) - throws Exception { + public CompletableFuture createJobMasterService( + UUID leaderSessionId, OnCompletionActions onCompletionActions) { + + return CompletableFuture.supplyAsync( + FunctionUtils.uncheckedSupplier( + () -> internalCreateJobMasterService(leaderSessionId, onCompletionActions)), + executor); + } + + private JobMasterService internalCreateJobMasterService( + UUID leaderSessionId, OnCompletionActions onCompletionActions) throws Exception { final JobMaster jobMaster = new JobMaster( rpcService, - jobMasterId, + JobMasterId.fromUuidOrNull(leaderSessionId), jobMasterConfiguration, ResourceID.generate(), jobGraph, @@ -98,7 +111,7 @@ public JobMaster createJobMasterService( jobManagerSharedServices, heartbeatServices, jobManagerJobMetricGroupFactory, - jobCompletionActions, + onCompletionActions, fatalErrorHandler, userCodeClassloader, shuffleMaster, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java new file mode 100644 index 0000000000000..3d060c5da08c6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.factories; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.DefaultJobMasterServiceProcess; +import org.apache.flink.runtime.jobmaster.JobMasterServiceProcess; + +import javax.annotation.Nullable; + +import java.util.UUID; + +public class DefaultJobMasterServiceProcessFactory implements JobMasterServiceProcessFactory { + + private final JobID jobId; + private final String jobName; + private final long initializationTimestamp; + + private final JobMasterServiceFactory jobMasterServiceFactory; + + public DefaultJobMasterServiceProcessFactory( + JobID jobId, + String jobName, + long initializationTimestamp, + JobMasterServiceFactory jobMasterServiceFactory) { + this.jobId = jobId; + this.jobName = jobName; + this.initializationTimestamp = initializationTimestamp; + this.jobMasterServiceFactory = jobMasterServiceFactory; + } + + @Override + public JobMasterServiceProcess create(UUID leaderSessionId) { + return new DefaultJobMasterServiceProcess( + jobId, + leaderSessionId, + jobMasterServiceFactory, + cause -> createArchivedExecutionGraph(JobStatus.FAILED, cause)); + } + + @Override + public JobID getJobId() { + return jobId; + } + + @Override + public ArchivedExecutionGraph createArchivedExecutionGraph( + JobStatus jobStatus, @Nullable Throwable cause) { + return ArchivedExecutionGraph.createFromInitializingJob( + jobId, jobName, jobStatus, cause, initializationTimestamp); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceFactory.java index ebc9fc0e7f4cb..3a8978788728f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceFactory.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -18,19 +18,23 @@ package org.apache.flink.runtime.jobmaster.factories; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterService; -/** Factory for a {@link JobMasterService}. */ +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** Factory for the {@link JobMasterService}. */ public interface JobMasterServiceFactory { - JobMasterService createJobMasterService( - JobGraph jobGraph, - JobMasterId jobMasterId, - OnCompletionActions jobCompletionActions, - ClassLoader userCodeClassloader, - long initializationTimestamp) - throws Exception; + /** + * Creates a new {@link JobMasterService} for the given leaderSessionId and onCompletionActions. + * + * @param leaderSessionId leaderSessionId for which to create a {@link JobMasterService} + * @param onCompletionActions onCompletionActions which are given to the created {@link + * JobMasterService} + * @return Future which contains the newly created {@link JobMasterService} + */ + CompletableFuture createJobMasterService( + UUID leaderSessionId, OnCompletionActions onCompletionActions); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceProcessFactory.java new file mode 100644 index 0000000000000..040f6a21a8cb3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/JobMasterServiceProcessFactory.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.factories; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobMasterServiceProcess; + +import javax.annotation.Nullable; + +import java.util.UUID; + +/** Factory for the {@link JobMasterServiceProcess}. */ +public interface JobMasterServiceProcessFactory { + + /** + * Create a new {@link JobMasterServiceProcess} for the given leaderSessionId. + * + * @param leaderSessionId leaderSessionId for which to create a {@link JobMasterServiceProcess} + * @return the newly created {@link JobMasterServiceProcess} + */ + JobMasterServiceProcess create(UUID leaderSessionId); + + /** + * Gets the {@link JobID} of the job for which this factory creates {@link + * JobMasterServiceProcess}. + */ + JobID getJobId(); + + /** + * Creates an {@link ArchivedExecutionGraph} for the job for which this factory creates {@link + * JobMasterServiceProcess} with the given jobStatus and failure cause. + * + * @param jobStatus jobStatus which the {@link ArchivedExecutionGraph} should have + * @param cause cause which the {@link ArchivedExecutionGraph} should be initialized with; null + * iff no failure cause + * @return created {@link ArchivedExecutionGraph} + */ + ArchivedExecutionGraph createArchivedExecutionGraph( + JobStatus jobStatus, @Nullable Throwable cause); +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java deleted file mode 100644 index 32ff226258f9c..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherJobTest.java +++ /dev/null @@ -1,410 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.core.testutils.CommonTestUtils; -import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobmaster.JobManagerRunner; -import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; -import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; -import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; -import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; -import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.webmonitor.JobDetails; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - -import static org.hamcrest.core.Is.is; -import static org.hamcrest.core.StringContains.containsString; -import static org.junit.Assert.assertThat; - -/** Test for the {@link DispatcherJob} class. */ -public class DispatcherJobTest extends TestLogger { - - private static final Time TIMEOUT = Time.seconds(10L); - - @Test - public void testStatusWhenInitializing() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - assertThat(dispatcherJob.isInitialized(), is(false)); - assertThat(dispatcherJob.getResultFuture().isDone(), is(false)); - assertJobStatus(dispatcherJob, JobStatus.INITIALIZING); - } - - @Test - public void testStatusWhenRunning() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - // finish initialization - testContext.setRunning(); - - assertJobStatus(dispatcherJob, JobStatus.RUNNING); - - // result future not done - assertThat(dispatcherJob.getResultFuture().isDone(), is(false)); - - assertThat(dispatcherJob.isInitialized(), is(true)); - } - - @Test - public void testStatusWhenJobFinished() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - // finish job - testContext.setRunning(); - testContext.finishJob(); - - assertJobStatus(dispatcherJob, JobStatus.FINISHED); - - // assert result future done - DispatcherJobResult result = dispatcherJob.getResultFuture().get(); - - assertThat( - result.getExecutionGraphInfo().getArchivedExecutionGraph().getState(), - is(JobStatus.FINISHED)); - } - - @Test - public void testStatusWhenCancellingWhileInitializing() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - assertJobStatus(dispatcherJob, JobStatus.INITIALIZING); - - CompletableFuture cancelFuture = dispatcherJob.cancel(TIMEOUT); - - assertThat(cancelFuture.isDone(), is(false)); - assertThat(dispatcherJob.isInitialized(), is(false)); - - assertJobStatus(dispatcherJob, JobStatus.CANCELLING); - - testContext.setRunning(); - testContext.finishCancellation(); - - // assert that cancel future completes - cancelFuture.get(); - - assertJobStatus(dispatcherJob, JobStatus.CANCELED); - assertThat(dispatcherJob.isInitialized(), is(true)); - // assert that the result future completes - assertThat( - dispatcherJob - .getResultFuture() - .get() - .getExecutionGraphInfo() - .getArchivedExecutionGraph() - .getState(), - is(JobStatus.CANCELED)); - } - - @Test - public void testStatusWhenCancellingWhileRunning() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - testContext.setRunning(); - CompletableFuture cancelFuture = dispatcherJob.cancel(TIMEOUT); - - assertJobStatus(dispatcherJob, JobStatus.CANCELLING); - testContext.finishCancellation(); - - cancelFuture.get(); - assertJobStatus(dispatcherJob, JobStatus.CANCELED); - assertThat( - dispatcherJob - .getResultFuture() - .get() - .getExecutionGraphInfo() - .getArchivedExecutionGraph() - .getState(), - is(JobStatus.CANCELED)); - } - - @Test - public void testStatusWhenCancellingWhileFailed() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - RuntimeException exception = - new RuntimeException("Artificial failure in runner initialization"); - testContext.failInitialization(exception); - - assertJobStatus(dispatcherJob, JobStatus.FAILED); - - CommonTestUtils.assertThrows( - "Artificial failure", - ExecutionException.class, - () -> dispatcherJob.cancel(TIMEOUT).get()); - - assertJobStatus(dispatcherJob, JobStatus.FAILED); - } - - @Test - public void testErrorWhileInitializing() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - // now fail - RuntimeException exception = - new RuntimeException("Artificial failure in runner initialization"); - testContext.failInitialization(exception); - - assertThat(dispatcherJob.isInitialized(), is(true)); - assertJobStatus(dispatcherJob, JobStatus.FAILED); - - ArchivedExecutionGraph aeg = - dispatcherJob - .getResultFuture() - .get() - .getExecutionGraphInfo() - .getArchivedExecutionGraph(); - assertThat( - aeg.getFailureInfo() - .getException() - .deserializeError(ClassLoader.getSystemClassLoader()), - is(exception)); - } - - @Test - public void testDispatcherJobResult() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - testContext.failInitialization( - new RuntimeException("Artificial failure in runner initialization")); - - DispatcherJobResult result = dispatcherJob.getResultFuture().get(); - assertThat(result.isInitializationFailure(), is(true)); - assertThat( - result.getExecutionGraphInfo().getArchivedExecutionGraph().getState(), - is(JobStatus.FAILED)); - assertThat( - result.getInitializationFailure().getMessage(), - containsString("Artificial failure")); - } - - @Test - public void testCloseWhileInitializingSuccessfully() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - CompletableFuture closeFuture = dispatcherJob.closeAsync(); - assertThat(closeFuture.isDone(), is(false)); - - // set job running, so that we can cancel it - testContext.setRunning(); - - // assert future completes now - closeFuture.get(); - - // ensure the result future is complete (how it completes is up to the JobManager) - CompletableFuture resultFuture = dispatcherJob.getResultFuture(); - CommonTestUtils.assertThrows( - "has not been finished", ExecutionException.class, resultFuture::get); - } - - @Test - public void testCloseWhileInitializingErroneously() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - CompletableFuture closeFuture = dispatcherJob.closeAsync(); - assertThat(closeFuture.isDone(), is(false)); - - testContext.failInitialization(new RuntimeException("fail")); - - // assert future completes now - closeFuture.get(); - - // ensure the result future is complete - dispatcherJob.getResultFuture().get(); - } - - @Test - public void testCloseWhileRunning() throws Exception { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - - // complete JobManager runner future to indicate to the DispatcherJob that the Runner has - // been initialized - testContext.setRunning(); - - CompletableFuture closeFuture = dispatcherJob.closeAsync(); - - closeFuture.get(); - - // result future should complete exceptionally. - CompletableFuture resultFuture = dispatcherJob.getResultFuture(); - CommonTestUtils.assertThrows( - "has not been finished", ExecutionException.class, resultFuture::get); - } - - @Test(expected = IllegalStateException.class) - public void testUnavailableJobMasterGateway() { - TestContext testContext = createTestContext(); - DispatcherJob dispatcherJob = testContext.getDispatcherJob(); - dispatcherJob.getJobMasterGateway(); - } - - private TestContext createTestContext() { - JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); - CompletableFuture jobManagerRunnerCompletableFuture = - new CompletableFuture<>(); - DispatcherJob dispatcherJob = - DispatcherJob.createFor( - jobManagerRunnerCompletableFuture, - jobGraph.getJobID(), - jobGraph.getName(), - System.currentTimeMillis()); - - return new TestContext(jobManagerRunnerCompletableFuture, dispatcherJob, jobGraph); - } - - private static class TestContext { - private final CompletableFuture jobManagerRunnerCompletableFuture; - private final DispatcherJob dispatcherJob; - private final JobGraph jobGraph; - private final TestingJobMasterGateway mockRunningJobMasterGateway; - private final CompletableFuture cancellationFuture; - - private JobStatus internalJobStatus = JobStatus.INITIALIZING; - private CompletableFuture resultFuture = new CompletableFuture<>(); - - public TestContext( - CompletableFuture jobManagerRunnerCompletableFuture, - DispatcherJob dispatcherJob, - JobGraph jobGraph) { - this.jobManagerRunnerCompletableFuture = jobManagerRunnerCompletableFuture; - this.dispatcherJob = dispatcherJob; - this.jobGraph = jobGraph; - - this.cancellationFuture = new CompletableFuture<>(); - this.mockRunningJobMasterGateway = - new TestingJobMasterGatewayBuilder() - .setRequestJobSupplier( - () -> - CompletableFuture.completedFuture( - new ExecutionGraphInfo( - ArchivedExecutionGraph - .createFromInitializingJob( - getJobID(), - "test", - internalJobStatus, - null, - 1337)))) - .setRequestJobDetailsSupplier( - () -> { - JobDetails jobDetails = - new JobDetails( - getJobID(), - "", - 0, - 0, - 0, - internalJobStatus, - 0, - new int[] {0, 0, 0, 0, 0, 0, 0, 0, 0}, - 0); - return CompletableFuture.completedFuture(jobDetails); - }) - // once JobManagerRunner is initialized, complete result future with - // CANCELLED AEG and ack cancellation. - .setCancelFunction( - () -> { - internalJobStatus = JobStatus.CANCELLING; - return cancellationFuture; - }) - .build(); - } - - public JobID getJobID() { - return jobGraph.getJobID(); - } - - public void failInitialization(Throwable ex) { - jobManagerRunnerCompletableFuture.completeExceptionally(ex); - } - - public DispatcherJob getDispatcherJob() { - return dispatcherJob; - } - - public void setRunning() { - internalJobStatus = JobStatus.RUNNING; - JobManagerRunner jobManagerRunner = - new TestingJobManagerRunner.Builder() - .setJobId(getJobID()) - .setBlockingTermination(false) - .setJobMasterGatewayFuture( - CompletableFuture.completedFuture(mockRunningJobMasterGateway)) - .setResultFuture(resultFuture) - .build(); - jobManagerRunnerCompletableFuture.complete(jobManagerRunner); - } - - public void finishJob() { - internalJobStatus = JobStatus.FINISHED; - resultFuture.complete( - JobManagerRunnerResult.forSuccess( - new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( - getJobID(), "test", JobStatus.FINISHED, null, 1337)))); - } - - public void finishCancellation() { - jobManagerRunnerCompletableFuture.thenAccept( - runner -> { - internalJobStatus = JobStatus.CANCELED; - runner.getResultFuture() - .complete( - JobManagerRunnerResult.forSuccess( - new ExecutionGraphInfo( - ArchivedExecutionGraph - .createFromInitializingJob( - getJobID(), - "test", - JobStatus.CANCELED, - null, - 1337)))); - cancellationFuture.complete(Acknowledge.get()); - }); - } - } - - private void assertJobStatus(DispatcherJob dispatcherJob, JobStatus expectedStatus) - throws Exception { - assertThat(dispatcherJob.requestJobDetails(TIMEOUT).get().getStatus(), is(expectedStatus)); - assertThat( - dispatcherJob.requestJob(TIMEOUT).get().getArchivedExecutionGraph().getState(), - is(expectedStatus)); - assertThat(dispatcherJob.requestJobStatus(TIMEOUT).get(), is(expectedStatus)); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 09d9c3c2c372b..eb04d9d747f4b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobServer; @@ -31,7 +32,7 @@ import org.apache.flink.runtime.blob.TestingBlobStore; import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; -import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -58,7 +59,6 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -294,17 +294,16 @@ public void testBlobServerCleanupWhenJobNotFinished() throws Exception { @Test public void testBlobServerCleanupWhenJobSubmissionFails() throws Exception { startDispatcher(new FailingJobManagerRunnerFactory(new FlinkException("Test exception"))); - dispatcherGateway.submitJob(jobGraph, timeout).get(); - - Optional maybeError = - dispatcherGateway.requestJobResult(jobId, timeout).get().getSerializedThrowable(); + final CompletableFuture submissionFuture = + dispatcherGateway.submitJob(jobGraph, timeout); - assertThat(maybeError.isPresent(), is(true)); - Throwable exception = maybeError.get().deserializeError(this.getClass().getClassLoader()); + try { + submissionFuture.get(); + fail("Job submission was expected to fail."); + } catch (ExecutionException ee) { + assertThat(ee, FlinkMatchers.containsCause(JobSubmissionException.class)); + } - assertThat( - ExceptionUtils.findThrowable(exception, JobExecutionException.class).isPresent(), - is(true)); assertThatHABlobsHaveBeenRemoved(); } 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 d19f5b451e35e..49d5187c839d8 100755 --- 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 @@ -47,10 +47,16 @@ import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterService; +import org.apache.flink.runtime.jobmaster.JobMasterServiceLeadershipRunner; import org.apache.flink.runtime.jobmaster.JobNotFinishedException; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.jobmaster.TestingJobMasterService; +import org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceProcessFactory; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -78,6 +84,7 @@ import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; import org.apache.flink.util.TimeUtils; import org.apache.flink.util.function.ThrowingRunnable; @@ -116,8 +123,10 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; -import static org.hamcrest.Matchers.empty; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; @@ -215,109 +224,6 @@ private TestingDispatcher createAndStartDispatcher( return dispatcher; } - private static final class InitializationTimestampCapturingJobManagerRunnerFactory - implements JobManagerRunnerFactory { - private final BlockingQueue initializationTimestampQueue; - - private InitializationTimestampCapturingJobManagerRunnerFactory( - BlockingQueue initializationTimestampQueue) { - this.initializationTimestampQueue = initializationTimestampQueue; - } - - @Override - public JobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) { - initializationTimestampQueue.offer(initializationTimestamp); - return new TestingJobManagerRunner.Builder().setJobId(jobGraph.getJobID()).build(); - } - } - - /** Builder for the TestingDispatcher. */ - public class TestingDispatcherBuilder { - - private Collection initialJobGraphs = Collections.emptyList(); - - private DispatcherBootstrapFactory dispatcherBootstrapFactory = - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); - - private HeartbeatServices heartbeatServices = DispatcherTest.this.heartbeatServices; - - private HighAvailabilityServices haServices = DispatcherTest.this.haServices; - - private JobManagerRunnerFactory jobManagerRunnerFactory = - DefaultJobManagerRunnerFactory.INSTANCE; - - private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; - - TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { - this.heartbeatServices = heartbeatServices; - return this; - } - - TestingDispatcherBuilder setHaServices(HighAvailabilityServices haServices) { - this.haServices = haServices; - return this; - } - - TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGraphs) { - this.initialJobGraphs = initialJobGraphs; - return this; - } - - TestingDispatcherBuilder setDispatcherBootstrapFactory( - DispatcherBootstrapFactory dispatcherBootstrapFactory) { - this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; - return this; - } - - TestingDispatcherBuilder setJobManagerRunnerFactory( - JobManagerRunnerFactory jobManagerRunnerFactory) { - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - return this; - } - - TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { - this.jobGraphWriter = jobGraphWriter; - return this; - } - - TestingDispatcher build() throws Exception { - TestingResourceManagerGateway resourceManagerGateway = - new TestingResourceManagerGateway(); - - final MemoryExecutionGraphInfoStore executionGraphInfoStore = - new MemoryExecutionGraphInfoStore(); - - return new TestingDispatcher( - rpcService, - DispatcherId.generate(), - initialJobGraphs, - dispatcherBootstrapFactory, - new DispatcherServices( - configuration, - haServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - executionGraphInfoStore, - testingFatalErrorHandlerResource.getFatalErrorHandler(), - VoidHistoryServerArchivist.INSTANCE, - null, - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - jobGraphWriter, - jobManagerRunnerFactory, - ForkJoinPool.commonPool())); - } - } - @After public void tearDown() throws Exception { if (dispatcher != null) { @@ -394,55 +300,44 @@ public void testJobSubmissionWithPartialResourceConfigured() throws Exception { @Test public void testNonBlockingJobSubmission() throws Exception { - final OneShotLatch latch = new OneShotLatch(); - dispatcher = - createAndStartDispatcher( - heartbeatServices, - haServices, - new BlockingJobManagerRunnerFactory(latch::await)); + JobManagerRunnerWithBlockingJobMasterFactory blockingJobMaster = + new JobManagerRunnerWithBlockingJobMasterFactory(); + dispatcher = createAndStartDispatcher(heartbeatServices, haServices, blockingJobMaster); DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - final JobGraph emptyJobGraph = JobGraphTestUtils.emptyJobGraph(); - JobID jobID = emptyJobGraph.getJobID(); - - dispatcherGateway.submitJob(emptyJobGraph, TIMEOUT).get(); + blockingJobMaster.waitForBlockingInit(); // ensure INITIALIZING status assertThat( - dispatcherGateway.requestJobStatus(jobID, TIMEOUT).get(), + dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), is(JobStatus.INITIALIZING)); // ensure correct JobDetails MultipleJobsDetails multiDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get(); assertEquals(1, multiDetails.getJobs().size()); - assertEquals(jobID, multiDetails.getJobs().iterator().next().getJobId()); + assertEquals(jobId, multiDetails.getJobs().iterator().next().getJobId()); - // submission has succeeded, let the initialization finish. - latch.trigger(); + // let the initialization finish. + blockingJobMaster.unblockJobMasterInitialization(); // ensure job is running CommonTestUtils.waitUntilCondition( - () -> dispatcherGateway.requestJobStatus(jobID, TIMEOUT).get() == JobStatus.RUNNING, + () -> dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get() == JobStatus.RUNNING, Deadline.fromNow(TimeUtils.toDuration(TIMEOUT)), 5L); } @Test public void testInvalidCallDuringInitialization() throws Exception { - final OneShotLatch latch = new OneShotLatch(); - dispatcher = - createAndStartDispatcher( - heartbeatServices, - haServices, - new BlockingJobManagerRunnerFactory(latch::await)); - + JobManagerRunnerWithBlockingJobMasterFactory blockingJobMaster = + new JobManagerRunnerWithBlockingJobMasterFactory(); + dispatcher = createAndStartDispatcher(heartbeatServices, haServices, blockingJobMaster); DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - final JobGraph emptyJobGraph = - JobGraphBuilder.newStreamingJobGraphBuilder().setJobId(jobId).build(); - - dispatcherGateway.submitJob(emptyJobGraph, TIMEOUT).get(); + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); assertThat( dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), @@ -457,17 +352,6 @@ public void testInvalidCallDuringInitialization() throws Exception { } catch (ExecutionException t) { assertTrue(t.getCause() instanceof UnavailableDispatcherOperationException); } - - // submission has succeeded, let the initialization finish. - latch.trigger(); - - // ensure job is running - CommonTestUtils.waitUntilCondition( - () -> - dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get() - == JobStatus.RUNNING, - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT)), - 5L); } @Test @@ -527,7 +411,15 @@ public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception final FlinkException testFailure = new FlinkException("Test failure"); testingJobManagerRunner.completeResultFuture( - JobManagerRunnerResult.forInitializationFailure(testFailure)); + JobManagerRunnerResult.forInitializationFailure( + new ExecutionGraphInfo( + ArchivedExecutionGraph.createFromInitializingJob( + jobId, + jobGraph.getName(), + JobStatus.FAILED, + testFailure, + 1L)), + testFailure)); // wait till job has failed dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); @@ -652,64 +544,6 @@ private URI createTestingSavepoint() throws IOException, URISyntaxException { return new URI(completedCheckpointStorageLocation.getExternalPointer()); } - /** - * Tests that we wait until the JobMaster has gained leader ship before sending requests to it. - * See FLINK-8887. - */ - @Test - public void testWaitingForJobMasterLeadership() throws Exception { - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(); - dispatcher = - createAndStartDispatcher( - heartbeatServices, haServices, testingJobManagerRunnerFactory); - - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - log.info("Job submission completed"); - - // try getting a blocking, non-initializing job status future in a retry-loop. - // In some CI environments, we can not guarantee that the job immediately leaves the - // INITIALIZING status - // after the jobMasterLeaderElectionService has been started. - CompletableFuture jobStatusFuture = null; - for (int i = 0; i < 5; i++) { - jobStatusFuture = dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT); - try { - JobStatus status = jobStatusFuture.get(10, TimeUnit.MILLISECONDS); - if (status == JobStatus.INITIALIZING) { - jobStatusFuture = null; - Thread.sleep(100); - } - } catch (TimeoutException ignored) { - break; // great, we have a blocking future - } - } - if (jobStatusFuture == null) { - fail("Unable to get a job status future blocked on leader election."); - } - - final TestingJobManagerRunner testingJobManagerRunner = - testingJobManagerRunnerFactory.takeCreatedJobManagerRunner(); - - // completing the JobMasterGatewayFuture means that the JobManagerRunner has confirmed the - // leadership - testingJobManagerRunner.completeJobMasterGatewayFuture( - new TestingJobMasterGatewayBuilder() - .setRequestJobSupplier( - () -> - CompletableFuture.completedFuture( - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setState(JobStatus.RUNNING) - .build()))) - .build()); - - assertThat(jobStatusFuture.get(), is(JobStatus.RUNNING)); - } - /** * Tests that the {@link Dispatcher} fails fatally if the recovered jobs cannot be started. See * FLINK-9097. @@ -738,7 +572,15 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { // Let the initialization of the JobManagerRunner fail testingJobManagerRunner.completeResultFuture( - JobManagerRunnerResult.forInitializationFailure(testException)); + JobManagerRunnerResult.forInitializationFailure( + new ExecutionGraphInfo( + ArchivedExecutionGraph.createFromInitializingJob( + jobId, + jobGraph.getName(), + JobStatus.FAILED, + testException, + 1L)), + testException)); final Throwable error = fatalErrorHandler @@ -753,92 +595,42 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { fatalErrorHandler.clearError(); } - /** - * Tests that a blocking {@link JobManagerRunner} creation, e.g. due to blocking FileSystem - * access, does not block the {@link Dispatcher}. - * - *

See FLINK-10314 - */ - @Test - public void testBlockingJobManagerRunner() throws Exception { - final OneShotLatch jobManagerRunnerCreationLatch = new OneShotLatch(); - dispatcher = - createAndStartDispatcher( - heartbeatServices, - haServices, - new BlockingJobManagerRunnerFactory(jobManagerRunnerCreationLatch::await)); - - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - assertThat( - dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), - is(JobStatus.INITIALIZING)); - - final CompletableFuture> metricQueryServiceAddressesFuture = - dispatcherGateway.requestMetricQueryServiceAddresses(Time.seconds(5L)); - - assertThat(metricQueryServiceAddressesFuture.get(), is(empty())); - - assertThat( - dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), - is(JobStatus.INITIALIZING)); - - jobManagerRunnerCreationLatch.trigger(); - } - /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ @Test public void testFailingJobManagerRunnerCleanup() throws Exception { final FlinkException testException = new FlinkException("Test exception."); final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(2); + BlockingJobManagerRunnerFactory blockingJobManagerRunnerFactory = + new BlockingJobManagerRunnerFactory( + () -> { + final Optional take = queue.take(); + final Exception exception = take.orElse(null); + + if (exception != null) { + throw exception; + } + }); dispatcher = createAndStartDispatcher( - heartbeatServices, - haServices, - new BlockingJobManagerRunnerFactory( - () -> { - final Optional take = queue.take(); - final Exception exception = take.orElse(null); - - if (exception != null) { - throw exception; - } - })); + heartbeatServices, haServices, blockingJobManagerRunnerFactory); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - assertThat( - dispatcherGateway.requestJobStatus(jobGraph.getJobID(), TIMEOUT).get(), - is(JobStatus.INITIALIZING)); - queue.offer(Optional.of(testException)); - - // wait till job is failed - dispatcherGateway.requestJobResult(jobGraph.getJobID(), TIMEOUT).get(); - - ArchivedExecutionGraph execGraph = - dispatcherGateway.requestJob(jobGraph.getJobID(), TIMEOUT).get(); - Assert.assertNotNull(execGraph.getFailureInfo()); - assertThat( - ExceptionUtils.findSerializedThrowable( - execGraph.getFailureInfo().getException(), - FlinkException.class, - this.getClass().getClassLoader()) - .isPresent(), - is(true)); - - // submit job again - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + try { + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + } catch (Throwable expectedException) { + assertThat(expectedException, containsCause(FlinkException.class)); + assertThat(expectedException, containsMessage(testException.getMessage())); + } // don't fail this time queue.offer(Optional.empty()); + // submit job again + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + blockingJobManagerRunnerFactory.setJobStatus(JobStatus.RUNNING); // Ensure job is running CommonTestUtils.waitUntilCondition( @@ -907,7 +699,9 @@ public void testJobSuspensionWhenDispatcherIsTerminated() throws Exception { public void testShutDownClusterShouldCompleteShutDownFuture() throws Exception { dispatcher = createAndStartDispatcher( - heartbeatServices, haServices, DefaultJobManagerRunnerFactory.INSTANCE); + heartbeatServices, + haServices, + JobMasterServiceLeadershipRunnerFactory.INSTANCE); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); @@ -969,10 +763,82 @@ public void testInitializationTimestampForwardedToJobManagerRunner() throws Exce assertThat(initializationTimestamp, greaterThan(0L)); } + private static class JobManagerRunnerWithBlockingJobMasterFactory + implements JobManagerRunnerFactory { + + private final JobMasterGateway jobMasterGateway; + private final AtomicReference currentJobStatus; + private final BlockingQueue> jobMasterServiceFutures; + private final OneShotLatch initLatch; + + private JobManagerRunnerWithBlockingJobMasterFactory() { + this.currentJobStatus = new AtomicReference<>(JobStatus.INITIALIZING); + this.jobMasterServiceFutures = new ArrayBlockingQueue<>(2); + this.initLatch = new OneShotLatch(); + this.jobMasterGateway = + new TestingJobMasterGatewayBuilder() + .setRequestJobSupplier( + () -> + CompletableFuture.completedFuture( + new ExecutionGraphInfo( + new ArchivedExecutionGraphBuilder() + .setState( + currentJobStatus.get()) + .build()))) + .build(); + } + + @Override + public JobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) + throws Exception { + + return new JobMasterServiceLeadershipRunner( + new DefaultJobMasterServiceProcessFactory( + jobGraph.getJobID(), + jobGraph.getName(), + initializationTimestamp, + new TestingJobMasterServiceFactory( + () -> { + initLatch.trigger(); + final CompletableFuture result = + new CompletableFuture<>(); + jobMasterServiceFutures.offer(result); + return result; + })), + highAvailabilityServices.getJobManagerLeaderElectionService( + jobGraph.getJobID()), + highAvailabilityServices.getRunningJobsRegistry(), + jobManagerServices + .getLibraryCacheManager() + .registerClassLoaderLease(jobGraph.getJobID()), + fatalErrorHandler); + } + + public void waitForBlockingInit() throws InterruptedException { + initLatch.await(); + } + + public void unblockJobMasterInitialization() throws InterruptedException { + final CompletableFuture future = jobMasterServiceFutures.take(); + future.complete(new TestingJobMasterService(jobMasterGateway)); + currentJobStatus.set(JobStatus.RUNNING); + } + } + private static final class BlockingJobManagerRunnerFactory extends TestingJobManagerRunnerFactory { @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; + private TestingJobManagerRunner testingRunner; BlockingJobManagerRunnerFactory( @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { @@ -993,7 +859,7 @@ public TestingJobManagerRunner createJobManagerRunner( throws Exception { jobManagerRunnerCreationLatch.run(); - TestingJobManagerRunner testingRunner = + this.testingRunner = super.createJobManagerRunner( jobGraph, configuration, @@ -1022,6 +888,110 @@ public TestingJobManagerRunner createJobManagerRunner( testingRunner.completeJobMasterGatewayFuture(testingJobMasterGateway); return testingRunner; } + + public void setJobStatus(JobStatus newStatus) { + Preconditions.checkState( + testingRunner != null, + "JobManagerRunner must be created before this method is available"); + this.testingRunner.setJobStatus(newStatus); + } + } + + private static final class InitializationTimestampCapturingJobManagerRunnerFactory + implements JobManagerRunnerFactory { + private final BlockingQueue initializationTimestampQueue; + + private InitializationTimestampCapturingJobManagerRunnerFactory( + BlockingQueue initializationTimestampQueue) { + this.initializationTimestampQueue = initializationTimestampQueue; + } + + @Override + public JobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) { + initializationTimestampQueue.offer(initializationTimestamp); + return new TestingJobManagerRunner.Builder().setJobId(jobGraph.getJobID()).build(); + } + } + + /** Builder for the TestingDispatcher. */ + public class TestingDispatcherBuilder { + + private Collection initialJobGraphs = Collections.emptyList(); + + private final DispatcherBootstrapFactory dispatcherBootstrapFactory = + (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); + + private HeartbeatServices heartbeatServices = DispatcherTest.this.heartbeatServices; + + private HighAvailabilityServices haServices = DispatcherTest.this.haServices; + + private JobManagerRunnerFactory jobManagerRunnerFactory = + JobMasterServiceLeadershipRunnerFactory.INSTANCE; + + private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + + TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + TestingDispatcherBuilder setHaServices(HighAvailabilityServices haServices) { + this.haServices = haServices; + return this; + } + + TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGraphs) { + this.initialJobGraphs = initialJobGraphs; + return this; + } + + TestingDispatcherBuilder setJobManagerRunnerFactory( + JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + return this; + } + + TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { + this.jobGraphWriter = jobGraphWriter; + return this; + } + + TestingDispatcher build() throws Exception { + TestingResourceManagerGateway resourceManagerGateway = + new TestingResourceManagerGateway(); + + final MemoryExecutionGraphInfoStore executionGraphInfoStore = + new MemoryExecutionGraphInfoStore(); + + return new TestingDispatcher( + rpcService, + DispatcherId.generate(), + initialJobGraphs, + dispatcherBootstrapFactory, + new DispatcherServices( + configuration, + haServices, + () -> CompletableFuture.completedFuture(resourceManagerGateway), + blobServer, + heartbeatServices, + executionGraphInfoStore, + testingFatalErrorHandlerResource.getFatalErrorHandler(), + VoidHistoryServerArchivist.INSTANCE, + null, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + jobGraphWriter, + jobManagerRunnerFactory, + ForkJoinPool.commonPool())); + } } private Tuple2 getBlockingJobGraphAndVertex() { @@ -1038,23 +1008,6 @@ private Tuple2 getBlockingJobGraphAndVertex() { blockingJobVertex); } - private static class FailingJobVertex extends JobVertex { - - private static final long serialVersionUID = 3218428829168840760L; - - private final Exception failure; - - private FailingJobVertex(String name, Exception failure) { - super(name); - this.failure = failure; - } - - @Override - public void initializeOnMaster(ClassLoader loader) throws Exception { - throw failure; - } - } - private static final class ExpectedJobIdJobManagerRunnerFactory implements JobManagerRunnerFactory { @@ -1084,7 +1037,7 @@ public JobManagerRunner createJobManagerRunner( createdJobManagerRunnerLatch.countDown(); - return DefaultJobManagerRunnerFactory.INSTANCE.createJobManagerRunner( + return JobMasterServiceLeadershipRunnerFactory.INSTANCE.createJobManagerRunner( jobGraph, configuration, rpcService, @@ -1100,7 +1053,7 @@ public JobManagerRunner createJobManagerRunner( private static class BlockingJobVertex extends JobVertex { private final OneShotLatch oneShotLatch = new OneShotLatch(); - public BlockingJobVertex(String name) { + private BlockingJobVertex(String name) { super(name); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java index 7ec2bd1f5fa70..b2876bc2de2e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/BlobsCleanupITCase.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.BlobServerOptions; @@ -29,6 +28,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; @@ -41,7 +41,6 @@ import org.apache.flink.runtime.testutils.MiniClusterResource; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -55,19 +54,17 @@ import java.io.File; import java.io.FilenameFilter; import java.net.InetSocketAddress; -import java.nio.file.NoSuchFileException; import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -181,23 +178,21 @@ private void testBlobServerCleanup(final TestCase testCase) throws Exception { jobGraph.addUserJarBlobKey(new PermanentBlobKey()); } - final JobSubmissionResult jobSubmissionResult = miniCluster.submitJob(jobGraph).get(); + final CompletableFuture submissionFuture = + miniCluster.submitJob(jobGraph); if (testCase == TestCase.JOB_SUBMISSION_FAILS) { - // Wait for submission to fail & check if exception is forwarded - Optional exception = - miniCluster.requestJobResult(jid).get().getSerializedThrowable(); - assertTrue(exception.isPresent()); - assertTrue( - ExceptionUtils.findThrowableSerializedAware( - exception.get(), - NoSuchFileException.class, - getClass().getClassLoader()) - .isPresent()); - - // check job status - assertThat(miniCluster.getJobStatus(jid).get(), is(JobStatus.FAILED)); + try { + submissionFuture.get(); + fail("Expected job submission failure."); + } catch (ExecutionException e) { + assertThat( + ExceptionUtils.findThrowable(e, JobSubmissionException.class).isPresent(), + is(true)); + } } else { + final JobSubmissionResult jobSubmissionResult = submissionFuture.get(); + assertThat(jobSubmissionResult.getJobID(), is(jid)); final CompletableFuture resultFuture = miniCluster.requestJobResult(jid); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java new file mode 100644 index 0000000000000..713aee5c19bbb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.client.JobInitializationException; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; +import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.apache.flink.core.testutils.FlinkMatchers.futureWillCompleteExceptionally; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Tests for the {@link DefaultJobMasterServiceProcess}. */ +public class DefaultJobMasterServiceProcessTest extends TestLogger { + private static final Duration TIMEOUT = Duration.ofSeconds(10); + private static final JobID jobId = new JobID(); + private static final Function + failedArchivedExecutionGraphFactory = + (throwable -> + ArchivedExecutionGraph.createFromInitializingJob( + jobId, "test", JobStatus.FAILED, throwable, 1337)); + + @Test + public void testInitializationFailureCompletesResultFuture() { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + final RuntimeException originalCause = new RuntimeException("Init error"); + jobMasterServiceFuture.completeExceptionally(originalCause); + + assertTrue(serviceProcess.getResultFuture().join().isInitializationFailure()); + final Throwable initializationFailure = + serviceProcess.getResultFuture().join().getInitializationFailure(); + assertThat(initializationFailure, containsCause(JobInitializationException.class)); + assertThat(initializationFailure, containsCause(originalCause)); + } + + @Test + public void testCloseAfterInitializationFailure() throws Exception { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + jobMasterServiceFuture.completeExceptionally(new RuntimeException("Init error")); + + serviceProcess.closeAsync().get(); + assertTrue(serviceProcess.getResultFuture().join().isInitializationFailure()); + assertThat(serviceProcess.getJobMasterGatewayFuture().isCompletedExceptionally(), is(true)); + } + + @Test + public void testCloseAfterInitializationSuccess() throws Exception { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + TestingJobMasterService testingJobMasterService = new TestingJobMasterService(); + jobMasterServiceFuture.complete(testingJobMasterService); + + serviceProcess.closeAsync().get(); + assertThat(testingJobMasterService.isClosed(), is(true)); + assertThat( + serviceProcess.getResultFuture(), + futureWillCompleteExceptionally(JobNotFinishedException.class, TIMEOUT)); + } + + @Test + public void testJobMasterTerminationIsHandled() { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + CompletableFuture jobMasterTerminationFuture = new CompletableFuture<>(); + TestingJobMasterService testingJobMasterService = + new TestingJobMasterService("localhost", jobMasterTerminationFuture, null); + jobMasterServiceFuture.complete(testingJobMasterService); + + RuntimeException testException = new RuntimeException("Fake exception from JobMaster"); + jobMasterTerminationFuture.completeExceptionally(testException); + + try { + serviceProcess.getResultFuture().get(); + fail("Expect failure"); + } catch (Throwable t) { + assertThat(t, containsCause(RuntimeException.class)); + assertThat(t, containsMessage(testException.getMessage())); + } + } + + @Test + public void testJobMasterGatewayGetsForwarded() throws Exception { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + TestingJobMasterGateway testingGateway = new TestingJobMasterGatewayBuilder().build(); + TestingJobMasterService testingJobMasterService = + new TestingJobMasterService("localhost", null, testingGateway); + jobMasterServiceFuture.complete(testingJobMasterService); + + assertThat(serviceProcess.getJobMasterGatewayFuture().get(), is(testingGateway)); + } + + @Test + public void testLeaderAddressGetsForwarded() throws Exception { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + String testingAddress = "yolohost"; + TestingJobMasterService testingJobMasterService = + new TestingJobMasterService(testingAddress, null, null); + jobMasterServiceFuture.complete(testingJobMasterService); + + assertThat(serviceProcess.getLeaderAddressFuture().get(), is(testingAddress)); + } + + @Test + public void testIsNotInitialized() { + DefaultJobMasterServiceProcess serviceProcess = + createTestInstance(new CompletableFuture<>()); + assertThat(serviceProcess.isInitializedAndRunning(), is(false)); + } + + @Test + public void testIsInitialized() { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + + jobMasterServiceFuture.complete(new TestingJobMasterService()); + + assertThat(serviceProcess.isInitializedAndRunning(), is(true)); + } + + @Test + public void testIsNotInitializedAfterClosing() { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + + jobMasterServiceFuture.complete(new TestingJobMasterService()); + + serviceProcess.closeAsync(); + + assertFalse(serviceProcess.isInitializedAndRunning()); + } + + @Test + public void testSuccessOnTerminalState() throws Exception { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + jobMasterServiceFuture.complete(new TestingJobMasterService()); + ArchivedExecutionGraph archivedExecutionGraph = + new ArchivedExecutionGraphBuilder().setState(JobStatus.FINISHED).build(); + + serviceProcess.jobReachedGloballyTerminalState( + new ExecutionGraphInfo(archivedExecutionGraph)); + + assertThat(serviceProcess.getResultFuture().get().isSuccess(), is(true)); + assertThat( + serviceProcess + .getResultFuture() + .get() + .getExecutionGraphInfo() + .getArchivedExecutionGraph() + .getState(), + is(JobStatus.FINISHED)); + } + + @Test + public void testJobFinishedByOther() { + final CompletableFuture jobMasterServiceFuture = + new CompletableFuture<>(); + DefaultJobMasterServiceProcess serviceProcess = createTestInstance(jobMasterServiceFuture); + jobMasterServiceFuture.complete(new TestingJobMasterService()); + + serviceProcess.jobFinishedByOther(); + + assertThat( + serviceProcess.getResultFuture(), + futureWillCompleteExceptionally(JobNotFinishedException.class, TIMEOUT)); + } + + private DefaultJobMasterServiceProcess createTestInstance( + CompletableFuture jobMasterServiceFuture) { + + return new DefaultJobMasterServiceProcess( + jobId, + UUID.randomUUID(), + new TestingJobMasterServiceFactory(() -> jobMasterServiceFuture), + failedArchivedExecutionGraphFactory); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java deleted file mode 100644 index 4f5c42e216b53..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerImplTest.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.jobmaster; - -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.core.testutils.FlinkMatchers; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.client.JobInitializationException; -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; -import org.apache.flink.runtime.execution.librarycache.TestingClassLoaderLease; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceFactory; -import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory; -import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; -import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; -import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.runtime.util.TestingUserCodeClassLoader; -import org.apache.flink.util.FlinkException; -import org.apache.flink.util.TestLogger; - -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import javax.annotation.Nonnull; - -import java.time.Duration; -import java.util.UUID; -import java.util.concurrent.CompletableFuture; -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.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** Tests for the {@link JobManagerRunnerImpl}. */ -public class JobManagerRunnerImplTest extends TestLogger { - - @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private static JobGraph jobGraph; - - private static ExecutionGraphInfo executionGraphInfo; - - private static JobMasterServiceFactory defaultJobMasterServiceFactory; - - private TestingHighAvailabilityServices haServices; - - private TestingLeaderElectionService leaderElectionService; - - private TestingFatalErrorHandler fatalErrorHandler; - - @BeforeClass - public static void setupClass() { - defaultJobMasterServiceFactory = new TestingJobMasterServiceFactory(); - - final JobVertex jobVertex = new JobVertex("Test vertex"); - jobVertex.setInvokableClass(NoOpInvokable.class); - jobGraph = JobGraphTestUtils.streamingJobGraph(jobVertex); - - executionGraphInfo = - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setJobID(jobGraph.getJobID()) - .setState(JobStatus.FINISHED) - .build()); - } - - @Before - public void setup() { - leaderElectionService = new TestingLeaderElectionService(); - haServices = new TestingHighAvailabilityServices(); - haServices.setJobMasterLeaderElectionService(jobGraph.getJobID(), leaderElectionService); - haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); - haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); - - fatalErrorHandler = new TestingFatalErrorHandler(); - } - - @After - public void tearDown() throws Exception { - fatalErrorHandler.rethrowError(); - } - - @Test - public void testJobCompletion() throws Exception { - final JobManagerRunnerImpl jobManagerRunner = createJobManagerRunner(); - - try { - jobManagerRunner.start(); - - final CompletableFuture resultFuture = - jobManagerRunner.getResultFuture(); - - assertThat(resultFuture.isDone(), is(false)); - - jobManagerRunner.jobReachedGloballyTerminalState(executionGraphInfo); - - final JobManagerRunnerResult jobManagerRunnerResult = resultFuture.get(); - assertThat( - jobManagerRunnerResult, - is(JobManagerRunnerResult.forSuccess(executionGraphInfo))); - } finally { - jobManagerRunner.close(); - } - } - - @Test - public void testJobFinishedByOther() throws Exception { - final JobManagerRunnerImpl jobManagerRunner = createJobManagerRunner(); - - try { - jobManagerRunner.start(); - - final CompletableFuture resultFuture = - jobManagerRunner.getResultFuture(); - - assertThat(resultFuture.isDone(), is(false)); - - jobManagerRunner.jobFinishedByOther(); - - final JobManagerRunnerResult jobManagerRunnerResult = resultFuture.get(); - - assertTrue(jobManagerRunnerResult.isJobNotFinished()); - } finally { - jobManagerRunner.close(); - } - } - - @Test - public void testShutDown() throws Exception { - final JobManagerRunner jobManagerRunner = createJobManagerRunner(); - - try { - jobManagerRunner.start(); - - final CompletableFuture resultFuture = - jobManagerRunner.getResultFuture(); - - assertThat(resultFuture.isDone(), is(false)); - - jobManagerRunner.closeAsync(); - - final JobManagerRunnerResult jobManagerRunnerResult = resultFuture.join(); - - assertTrue(jobManagerRunnerResult.isJobNotFinished()); - } finally { - jobManagerRunner.close(); - } - } - - @Test - public void testLibraryCacheManagerRegistration() throws Exception { - final OneShotLatch registerClassLoaderLatch = new OneShotLatch(); - final OneShotLatch closeClassLoaderLeaseLatch = new OneShotLatch(); - final TestingUserCodeClassLoader userCodeClassLoader = - TestingUserCodeClassLoader.newBuilder().build(); - final TestingClassLoaderLease classLoaderLease = - TestingClassLoaderLease.newBuilder() - .setGetOrResolveClassLoaderFunction( - (permanentBlobKeys, urls) -> { - registerClassLoaderLatch.trigger(); - return userCodeClassLoader; - }) - .setCloseRunnable(closeClassLoaderLeaseLatch::trigger) - .build(); - final JobManagerRunner jobManagerRunner = createJobManagerRunner(classLoaderLease); - - try { - jobManagerRunner.start(); - - registerClassLoaderLatch.await(); - - jobManagerRunner.close(); - - closeClassLoaderLeaseLatch.await(); - } finally { - jobManagerRunner.close(); - } - } - - /** - * Tests that the {@link JobManagerRunnerImpl} always waits for the previous leadership - * operation (granting or revoking leadership) to finish before starting a new leadership - * operation. - */ - @Test - public void testConcurrentLeadershipOperationsBlockingClose() throws Exception { - final CompletableFuture terminationFuture = new CompletableFuture<>(); - - TestingJobMasterServiceFactory jobMasterServiceFactory = - new TestingJobMasterServiceFactory( - () -> new TestingJobMasterService("localhost", terminationFuture)); - JobManagerRunner jobManagerRunner = createJobManagerRunner(jobMasterServiceFactory); - - jobManagerRunner.start(); - - leaderElectionService.isLeader(UUID.randomUUID()).get(); - - leaderElectionService.notLeader(); - - final CompletableFuture leaderFuture = - leaderElectionService.isLeader(UUID.randomUUID()); - - // the new leadership should wait first for the suspension to happen - assertThat(leaderFuture.isDone(), is(false)); - - try { - leaderFuture.get(1L, TimeUnit.MILLISECONDS); - fail("Granted leadership even though the JobMaster has not been suspended."); - } catch (TimeoutException expected) { - // expected - } - - terminationFuture.complete(null); - - leaderFuture.get(); - } - - @Test - public void testJobMasterServiceTerminatesUnexpectedlyTriggersFailure() throws Exception { - final CompletableFuture terminationFuture = new CompletableFuture<>(); - - TestingJobMasterServiceFactory jobMasterServiceFactory = - new TestingJobMasterServiceFactory( - () -> new TestingJobMasterService("localhost", terminationFuture)); - JobManagerRunner jobManagerRunner = createJobManagerRunner(jobMasterServiceFactory); - - jobManagerRunner.start(); - - leaderElectionService.isLeader(UUID.randomUUID()).get(); - - terminationFuture.completeExceptionally( - new FlinkException("The JobMasterService failed unexpectedly.")); - - assertThat( - jobManagerRunner.getResultFuture(), - FlinkMatchers.futureWillCompleteExceptionally(Duration.ofSeconds(10L))); - } - - @Test - public void testJobMasterCreationFailureCompletesJobManagerRunnerWithInitializationError() - throws Exception { - - final FlinkException testException = new FlinkException("Test exception"); - final TestingJobMasterServiceFactory jobMasterServiceFactory = - new TestingJobMasterServiceFactory( - () -> { - throw testException; - }); - - final JobManagerRunner jobManagerRunner = createJobManagerRunner(jobMasterServiceFactory); - - jobManagerRunner.start(); - - leaderElectionService.isLeader(UUID.randomUUID()); - - final JobManagerRunnerResult jobManagerRunnerResult = - jobManagerRunner.getResultFuture().join(); - assertTrue(jobManagerRunnerResult.isInitializationFailure()); - assertTrue( - jobManagerRunnerResult.getInitializationFailure() - instanceof JobInitializationException); - assertThat( - jobManagerRunnerResult.getInitializationFailure(), - FlinkMatchers.containsCause(testException)); - } - - @Nonnull - private JobManagerRunner createJobManagerRunner( - LibraryCacheManager.ClassLoaderLease classLoaderLease) throws Exception { - return createJobManagerRunner(defaultJobMasterServiceFactory, classLoaderLease); - } - - @Nonnull - private JobManagerRunnerImpl createJobManagerRunner() throws Exception { - return createJobManagerRunner( - defaultJobMasterServiceFactory, TestingClassLoaderLease.newBuilder().build()); - } - - @Nonnull - private JobManagerRunner createJobManagerRunner(JobMasterServiceFactory jobMasterServiceFactory) - throws Exception { - return createJobManagerRunner( - jobMasterServiceFactory, TestingClassLoaderLease.newBuilder().build()); - } - - @Nonnull - private JobManagerRunnerImpl createJobManagerRunner( - JobMasterServiceFactory jobMasterServiceFactory, - LibraryCacheManager.ClassLoaderLease classLoaderLease) - throws Exception { - return new JobManagerRunnerImpl( - jobGraph, - jobMasterServiceFactory, - haServices, - classLoaderLease, - TestingUtils.defaultExecutor(), - fatalErrorHandler, - System.currentTimeMillis()); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java index d90d753b97bb7..f7a43e6070ee9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerResultTest.java @@ -43,28 +43,16 @@ public void testSuccessfulJobManagerResult() { JobManagerRunnerResult.forSuccess(executionGraphInfo); assertTrue(jobManagerRunnerResult.isSuccess()); - assertFalse(jobManagerRunnerResult.isJobNotFinished()); - assertFalse(jobManagerRunnerResult.isInitializationFailure()); - } - - @Test - public void testJobNotFinishedJobManagerResult() { - final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forJobNotFinished(); - - assertTrue(jobManagerRunnerResult.isJobNotFinished()); - assertFalse(jobManagerRunnerResult.isSuccess()); assertFalse(jobManagerRunnerResult.isInitializationFailure()); } @Test public void testInitializationFailureJobManagerResult() { final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forInitializationFailure(testException); + JobManagerRunnerResult.forInitializationFailure(executionGraphInfo, testException); assertTrue(jobManagerRunnerResult.isInitializationFailure()); assertFalse(jobManagerRunnerResult.isSuccess()); - assertFalse(jobManagerRunnerResult.isJobNotFinished()); } @Test @@ -75,38 +63,14 @@ public void testGetArchivedExecutionGraphFromSuccessfulJobManagerResult() { assertThat(jobManagerRunnerResult.getExecutionGraphInfo(), is(executionGraphInfo)); } - @Test(expected = IllegalStateException.class) - public void testGetArchivedExecutionGraphFromJobNotFinishedFails() { - final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forJobNotFinished(); - - jobManagerRunnerResult.getExecutionGraphInfo(); - } - - @Test(expected = IllegalStateException.class) - public void testGetArchivedExecutionGraphFromInitializationFailureFails() { - final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forInitializationFailure(testException); - - jobManagerRunnerResult.getExecutionGraphInfo(); - } - @Test public void testGetInitializationFailureFromFailedJobManagerResult() { final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forInitializationFailure(testException); + JobManagerRunnerResult.forInitializationFailure(executionGraphInfo, testException); assertThat(jobManagerRunnerResult.getInitializationFailure(), is(testException)); } - @Test(expected = IllegalStateException.class) - public void testGetInitializationFailureFromJobNotFinished() { - final JobManagerRunnerResult jobManagerRunnerResult = - JobManagerRunnerResult.forJobNotFinished(); - - jobManagerRunnerResult.getInitializationFailure(); - } - @Test(expected = IllegalStateException.class) public void testGetInitializationFailureFromSuccessfulJobManagerResult() { final JobManagerRunnerResult jobManagerRunnerResult = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java new file mode 100644 index 0000000000000..c00cdfb19fc17 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java @@ -0,0 +1,719 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.core.testutils.FlinkMatchers; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; +import org.apache.flink.runtime.execution.librarycache.TestingClassLoaderLease; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; +import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceProcessFactory; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nonnull; + +import java.time.Duration; +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Queue; +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 java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.apache.flink.core.testutils.FlinkMatchers.willNotComplete; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** Tests for the {@link JobMasterServiceLeadershipRunner}. */ +public class JobMasterServiceLeadershipRunnerTest extends TestLogger { + + private static final Time TESTING_TIMEOUT = Time.seconds(10); + + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static JobGraph jobGraph; + + private TestingLeaderElectionService leaderElectionService; + + private TestingFatalErrorHandler fatalErrorHandler; + + private RunningJobsRegistry runningJobsRegistry; + + @BeforeClass + public static void setupClass() { + + final JobVertex jobVertex = new JobVertex("Test vertex"); + jobVertex.setInvokableClass(NoOpInvokable.class); + jobGraph = JobGraphTestUtils.streamingJobGraph(jobVertex); + } + + @Before + public void setup() { + leaderElectionService = new TestingLeaderElectionService(); + runningJobsRegistry = new StandaloneRunningJobsRegistry(); + fatalErrorHandler = new TestingFatalErrorHandler(); + } + + @After + public void tearDown() throws Exception { + fatalErrorHandler.rethrowError(); + } + + @Test + public void testShutDownSignalsJobAsNotFinished() throws Exception { + try (JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder().build()) { + jobManagerRunner.start(); + + final CompletableFuture resultFuture = + jobManagerRunner.getResultFuture(); + + assertThat(resultFuture.isDone(), is(false)); + + jobManagerRunner.closeAsync(); + + assertJobNotFinished(resultFuture); + assertThat( + jobManagerRunner.getJobMasterGateway(), + FlinkMatchers.futureWillCompleteExceptionally(Duration.ofMillis(5L))); + } + } + + @Test + public void testCloseReleasesClassLoaderLease() throws Exception { + final OneShotLatch closeClassLoaderLeaseLatch = new OneShotLatch(); + + final TestingClassLoaderLease classLoaderLease = + TestingClassLoaderLease.newBuilder() + .setCloseRunnable(closeClassLoaderLeaseLatch::trigger) + .build(); + + try (JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .setClassLoaderLease(classLoaderLease) + .build()) { + jobManagerRunner.start(); + + jobManagerRunner.close(); + + closeClassLoaderLeaseLatch.await(); + } + } + + /** + * Tests that the {@link JobMasterServiceLeadershipRunner} always waits for the previous + * leadership operation (granting or revoking leadership) to finish before starting a new + * leadership operation. + */ + @Test + public void testConcurrentLeadershipOperationsBlockingClose() throws Exception { + final CompletableFuture terminationFuture = new CompletableFuture<>(); + + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withJobMasterServiceProcesses( + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(terminationFuture) + .withManualTerminationFutureCompletion() + .build(), + TestingJobMasterServiceProcess.newBuilder().build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()).get(); + + leaderElectionService.notLeader(); + + final CompletableFuture leaderFuture = + leaderElectionService.isLeader(UUID.randomUUID()); + + // the new leadership should wait first for the suspension to happen + assertThat(leaderFuture.isDone(), is(false)); + + try { + leaderFuture.get(1L, TimeUnit.MILLISECONDS); + fail("Granted leadership even though the JobMaster has not been suspended."); + } catch (TimeoutException expected) { + // expected + } + + terminationFuture.complete(null); + + leaderFuture.get(); + } + + @Test + public void testExceptionallyCompletedResultFutureFromJobMasterServiceProcessIsForwarded() + throws Exception { + final CompletableFuture resultFuture = new CompletableFuture<>(); + final TestingJobMasterServiceProcess testingJobMasterServiceProcess = + TestingJobMasterServiceProcess.newBuilder() + .setJobManagerRunnerResultFuture(resultFuture) + .build(); + + JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess(testingJobMasterServiceProcess) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()).get(); + + final FlinkException cause = + new FlinkException("The JobMasterService failed unexpectedly."); + resultFuture.completeExceptionally(cause); + + assertThat( + jobManagerRunner.getResultFuture(), + FlinkMatchers.futureWillCompleteExceptionally( + cause::equals, + Duration.ofMillis(5L), + "Wrong cause of failed result future")); + } + + @Test + public void testJobMasterCreationFailureCompletesJobManagerRunnerWithInitializationError() + throws Exception { + + final FlinkException testException = new FlinkException("Test exception"); + + final CompletableFuture completedResultFuture = + CompletableFuture.completedFuture( + JobManagerRunnerResult.forInitializationFailure( + createFailedExecutionGraphInfo(testException), testException)); + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setJobManagerRunnerResultFuture(completedResultFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + final JobManagerRunnerResult jobManagerRunnerResult = + jobManagerRunner.getResultFuture().join(); + assertTrue(jobManagerRunnerResult.isInitializationFailure()); + + assertThat(jobManagerRunnerResult.getInitializationFailure(), containsCause(testException)); + } + + @Nonnull + private ExecutionGraphInfo createFailedExecutionGraphInfo(FlinkException testException) { + return new ExecutionGraphInfo( + ArchivedExecutionGraph.createFromInitializingJob( + jobGraph.getJobID(), + jobGraph.getName(), + JobStatus.FAILED, + testException, + 1L)); + } + + @Test + public void testJobMasterServiceProcessIsTerminatedOnClose() throws Exception { + final CompletableFuture terminationFuture = new CompletableFuture<>(); + + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(terminationFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + jobManagerRunner.closeAsync().join(); + + assertJobNotFinished(jobManagerRunner.getResultFuture()); + assertTrue(terminationFuture.isDone()); + } + + @Test + public void testJobMasterServiceProcessShutdownOnLeadershipLoss() throws Exception { + final CompletableFuture terminationFuture = new CompletableFuture<>(); + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(terminationFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + leaderElectionService.notLeader(); + + assertTrue(terminationFuture.isDone()); + } + + @Test + public void testCancellationIsForwardedToJobMasterService() throws Exception { + final CompletableFuture jobMasterGatewayFuture = + new CompletableFuture<>(); + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setJobMasterGatewayFuture(jobMasterGatewayFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + // cancel during init + CompletableFuture cancellationFuture = + jobManagerRunner.cancel(TESTING_TIMEOUT); + + assertThat(cancellationFuture.isDone(), is(false)); + + AtomicBoolean cancelCalled = new AtomicBoolean(false); + JobMasterGateway jobMasterGateway = + new TestingJobMasterGatewayBuilder() + .setCancelFunction( + () -> { + cancelCalled.set(true); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + jobMasterGatewayFuture.complete(jobMasterGateway); + + // assert that cancellation future completes when cancellation completes. + cancellationFuture.get(); + assertThat(cancelCalled.get(), is(true)); + } + + @Test + public void testJobInformationOperationsDuringInitialization() throws Exception { + + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setIsInitialized(false) + .build()) + .build(); + + jobManagerRunner.start(); + + // assert initializing while waiting for leadership + assertInitializingStates(jobManagerRunner); + + // assign leadership + leaderElectionService.isLeader(UUID.randomUUID()); + + // assert initializing while not yet initialized + assertInitializingStates(jobManagerRunner); + } + + private static void assertInitializingStates(JobManagerRunner jobManagerRunner) + throws ExecutionException, InterruptedException { + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).get(), + is(JobStatus.INITIALIZING)); + assertThat(jobManagerRunner.getResultFuture().isDone(), is(false)); + assertThat( + jobManagerRunner + .requestJob(TESTING_TIMEOUT) + .get() + .getArchivedExecutionGraph() + .getState(), + is(JobStatus.INITIALIZING)); + + assertThat( + jobManagerRunner.requestJobDetails(TESTING_TIMEOUT).get().getStatus(), + is(JobStatus.INITIALIZING)); + } + + // It can happen that a series of leadership operations happens while the JobMaster + // initialization is blocked. This test is to ensure that we are not starting-stopping + // JobMasters for all pending leadership grants, but only for the latest. + @Test + public void testSkippingOfEnqueuedLeadershipOperations() throws Exception { + final CompletableFuture firstTerminationFuture = new CompletableFuture<>(); + final CompletableFuture secondTerminationFuture = new CompletableFuture<>(); + + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withJobMasterServiceProcesses( + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(firstTerminationFuture) + .withManualTerminationFutureCompletion() + .setIsInitialized(false) + .build(), + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(secondTerminationFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + // first leadership assignment to get into blocking initialization + leaderElectionService.isLeader(UUID.randomUUID()); + + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).get(), + is(JobStatus.INITIALIZING)); + + // we are now blocked on the initialization, enqueue some operations: + for (int i = 0; i < 10; i++) { + leaderElectionService.notLeader(); + leaderElectionService.isLeader(UUID.randomUUID()); + } + + firstTerminationFuture.complete(null); + + jobManagerRunner.closeAsync(); + + // this ensures that the second JobMasterServiceProcess is taken + assertTrue(secondTerminationFuture.isDone()); + } + + @Test + public void testCancellationFailsWhenInitializationFails() throws Exception { + final FlinkException testException = new FlinkException("test exception"); + runCancellationFailsTest( + resultFuture -> + resultFuture.complete( + JobManagerRunnerResult.forInitializationFailure( + createFailedExecutionGraphInfo(testException), + testException))); + } + + @Test + public void testCancellationFailsWhenExceptionOccurs() throws Exception { + final FlinkException testException = new FlinkException("test exception"); + runCancellationFailsTest(resultFuture -> resultFuture.completeExceptionally(testException)); + } + + public void runCancellationFailsTest( + Consumer> testAction) throws Exception { + final CompletableFuture jobManagerRunnerResultFuture = + new CompletableFuture<>(); + final JobManagerRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setIsInitialized(false) + .setJobMasterGatewayFuture(new CompletableFuture<>()) + .setJobManagerRunnerResultFuture( + jobManagerRunnerResultFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + // cancel while initializing + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).get(), + is(JobStatus.INITIALIZING)); + + CompletableFuture cancelFuture = jobManagerRunner.cancel(TESTING_TIMEOUT); + assertThat(cancelFuture.isDone(), is(false)); + + testAction.accept(jobManagerRunnerResultFuture); + + try { + cancelFuture.get(); + fail(); + } catch (Throwable t) { + assertThat(t, containsMessage("Cancellation failed.")); + } + } + + @Test + public void testResultFutureCompletionOfOutdatedLeaderIsIgnored() throws Exception { + final CompletableFuture resultFuture = new CompletableFuture<>(); + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setJobManagerRunnerResultFuture(resultFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()).join(); + + leaderElectionService.notLeader(); + + resultFuture.complete( + JobManagerRunnerResult.forSuccess( + createFailedExecutionGraphInfo(new FlinkException("test exception")))); + + assertThat(jobManagerRunner.getResultFuture(), willNotComplete(Duration.ofMillis(5L))); + } + + @Test + public void testJobMasterGatewayIsInvalidatedOnLeadershipChanges() throws Exception { + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setJobMasterGatewayFuture(new CompletableFuture<>()) + .build()) + .build(); + + jobManagerRunner.start(); + + final CompletableFuture jobMasterGateway = + jobManagerRunner.getJobMasterGateway(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + leaderElectionService.notLeader(); + + assertThat( + jobMasterGateway, + FlinkMatchers.futureWillCompleteExceptionally(Duration.ofMillis(5L))); + } + + @Test + public void testLeaderAddressOfOutdatedLeaderIsIgnored() throws Exception { + final CompletableFuture leaderAddressFuture = new CompletableFuture<>(); + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setLeaderAddressFuture(leaderAddressFuture) + .build()) + .build(); + + jobManagerRunner.start(); + + final CompletableFuture leaderFuture = + leaderElectionService.isLeader(UUID.randomUUID()); + + leaderElectionService.notLeader(); + + leaderAddressFuture.complete("foobar"); + + assertThat(leaderFuture, willNotComplete(Duration.ofMillis(5L))); + } + + @Test + public void testInitialJobStatusIsInitializing() throws Exception { + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder().build(); + + jobManagerRunner.start(); + + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).join(), + is(JobStatus.INITIALIZING)); + } + + @Test + public void testCancellationChangesJobStatusToCancelling() throws Exception { + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder().build(); + + jobManagerRunner.start(); + + jobManagerRunner.cancel(TESTING_TIMEOUT); + + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).join(), + is(JobStatus.CANCELLING)); + } + + @Test + public void testJobStatusCancellingIsClearedOnLeadershipLoss() throws Exception { + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder().build(); + + jobManagerRunner.start(); + + jobManagerRunner.cancel(TESTING_TIMEOUT); + + leaderElectionService.isLeader(UUID.randomUUID()); + leaderElectionService.notLeader(); + + assertThat( + jobManagerRunner.requestJobStatus(TESTING_TIMEOUT).join(), + is(JobStatus.INITIALIZING)); + } + + @Test + public void testJobMasterServiceProcessClosingExceptionIsForwardedToResultFuture() + throws Exception { + final CompletableFuture terminationFuture = new CompletableFuture<>(); + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .withSingleJobMasterServiceProcess( + TestingJobMasterServiceProcess.newBuilder() + .setTerminationFuture(terminationFuture) + .withManualTerminationFutureCompletion() + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + leaderElectionService.notLeader(); + + final FlinkException testException = new FlinkException("Test exception"); + terminationFuture.completeExceptionally(testException); + + assertThat( + jobManagerRunner.getResultFuture(), + FlinkMatchers.futureWillCompleteExceptionally( + cause -> + ExceptionUtils.findThrowable(cause, testException::equals) + .isPresent(), + Duration.ofMillis(5L), + "Result future should be completed exceptionally.")); + } + + @Test + public void testJobMasterServiceProcessCreationFailureIsForwardedToResultFuture() + throws Exception { + final FlinkRuntimeException testException = new FlinkRuntimeException("Test exception"); + final JobMasterServiceLeadershipRunner jobManagerRunner = + newJobMasterServiceLeadershipRunnerBuilder() + .setJobMasterServiceProcessFactory( + TestingJobMasterServiceProcessFactory.newBuilder() + .setJobMasterServiceProcessFunction( + ignored -> { + throw testException; + }) + .build()) + .build(); + + jobManagerRunner.start(); + + leaderElectionService.isLeader(UUID.randomUUID()); + + assertThat( + jobManagerRunner.getResultFuture(), + FlinkMatchers.futureWillCompleteExceptionally( + cause -> + ExceptionUtils.findThrowable(cause, testException::equals) + .isPresent(), + Duration.ofMillis(5L), + "Result future should be completed exceptionally.")); + } + + private void assertJobNotFinished(CompletableFuture resultFuture) { + try { + resultFuture.get(); + fail("Expect exception"); + } catch (Throwable t) { + assertThat(t, containsCause(JobNotFinishedException.class)); + } + } + + public JobMasterServiceLeadershipRunnerBuilder newJobMasterServiceLeadershipRunnerBuilder() { + return new JobMasterServiceLeadershipRunnerBuilder(); + } + + public class JobMasterServiceLeadershipRunnerBuilder { + private JobMasterServiceProcessFactory jobMasterServiceProcessFactory = + TestingJobMasterServiceProcessFactory.newBuilder().build(); + private LibraryCacheManager.ClassLoaderLease classLoaderLease = + TestingClassLoaderLease.newBuilder().build(); + + public JobMasterServiceLeadershipRunnerBuilder setClassLoaderLease( + LibraryCacheManager.ClassLoaderLease classLoaderLease) { + this.classLoaderLease = classLoaderLease; + return this; + } + + public JobMasterServiceLeadershipRunnerBuilder setJobMasterServiceProcessFactory( + JobMasterServiceProcessFactory jobMasterServiceProcessFactory) { + this.jobMasterServiceProcessFactory = jobMasterServiceProcessFactory; + return this; + } + + public JobMasterServiceLeadershipRunner build() { + return new JobMasterServiceLeadershipRunner( + jobMasterServiceProcessFactory, + leaderElectionService, + runningJobsRegistry, + classLoaderLease, + fatalErrorHandler); + } + + public JobMasterServiceLeadershipRunnerBuilder withSingleJobMasterServiceProcess( + JobMasterServiceProcess jobMasterServiceProcess) { + return withJobMasterServiceProcesses(jobMasterServiceProcess); + } + + public JobMasterServiceLeadershipRunnerBuilder withJobMasterServiceProcesses( + JobMasterServiceProcess... jobMasterServiceProcesses) { + final Queue jobMasterServiceProcessQueue = + new ArrayDeque<>(Arrays.asList(jobMasterServiceProcesses)); + this.jobMasterServiceProcessFactory = + TestingJobMasterServiceProcessFactory.newBuilder() + .setJobMasterServiceProcessFunction( + ignored -> + Preconditions.checkNotNull( + jobMasterServiceProcessQueue.poll())) + .build(); + return this; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java index 38e98613fb289..e7440c0b0163f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java @@ -19,7 +19,11 @@ package org.apache.flink.runtime.jobmaster; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.util.Preconditions; @@ -40,6 +44,8 @@ public class TestingJobManagerRunner implements JobManagerRunner { private final OneShotLatch closeAsyncCalledLatch = new OneShotLatch(); + private JobStatus jobStatus = JobStatus.INITIALIZING; + private TestingJobManagerRunner( JobID jobId, boolean blockingTermination, @@ -74,6 +80,31 @@ public JobID getJobID() { return jobId; } + @Override + public CompletableFuture cancel(Time timeout) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture requestJobStatus(Time timeout) { + return CompletableFuture.completedFuture(jobStatus); + } + + @Override + public CompletableFuture requestJobDetails(Time timeout) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture requestJob(Time timeout) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isInitialized() { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture closeAsync() { if (!blockingTermination) { @@ -84,6 +115,10 @@ public CompletableFuture closeAsync() { return terminationFuture; } + public void setJobStatus(JobStatus newStatus) { + this.jobStatus = newStatus; + } + public OneShotLatch getCloseAsyncCalledLatch() { return closeAsyncCalledLatch; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterService.java index 0cc2b6b9e08fe..27887f99013a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterService.java @@ -38,10 +38,16 @@ public class TestingJobMasterService implements JobMasterService { private final boolean completeTerminationFutureOnCloseAsync; public TestingJobMasterService( - @Nonnull String address, @Nullable CompletableFuture terminationFuture) { + @Nonnull String address, + @Nullable CompletableFuture terminationFuture, + @Nullable JobMasterGateway gateway) { this.address = address; - jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + if (gateway == null) { + jobMasterGateway = new TestingJobMasterGatewayBuilder().build(); + } else { + jobMasterGateway = gateway; + } if (terminationFuture == null) { this.terminationFuture = new CompletableFuture<>(); @@ -52,8 +58,12 @@ public TestingJobMasterService( } } + public TestingJobMasterService(JobMasterGateway gateway) { + this("localhost", null, gateway); + } + public TestingJobMasterService() { - this("localhost", null); + this("localhost", null, null); } @Override @@ -78,7 +88,10 @@ public CompletableFuture closeAsync() { if (completeTerminationFutureOnCloseAsync) { terminationFuture.complete(null); } - return terminationFuture; } + + public boolean isClosed() { + return terminationFuture.isDone(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterServiceProcess.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterServiceProcess.java new file mode 100644 index 0000000000000..fd3ae35033e44 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobMasterServiceProcess.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; + +/** Testing implementation of {@link JobMasterServiceProcess}. */ +public class TestingJobMasterServiceProcess implements JobMasterServiceProcess { + + private final CompletableFuture jobMasterGatewayFuture; + + private final CompletableFuture jobManagerRunnerResultFuture; + + private final CompletableFuture leaderAddressFuture; + + private final boolean isInitialized; + + private final CompletableFuture terminationFuture; + + private final boolean manualTerminationFutureCompletion; + + private TestingJobMasterServiceProcess( + CompletableFuture jobMasterGatewayFuture, + CompletableFuture jobManagerRunnerResultFuture, + CompletableFuture leaderAddressFuture, + boolean isInitialized, + CompletableFuture terminationFuture, + boolean manualTerminationFutureCompletion) { + this.jobMasterGatewayFuture = jobMasterGatewayFuture; + this.jobManagerRunnerResultFuture = jobManagerRunnerResultFuture; + this.leaderAddressFuture = leaderAddressFuture; + this.isInitialized = isInitialized; + this.terminationFuture = terminationFuture; + this.manualTerminationFutureCompletion = manualTerminationFutureCompletion; + } + + @Override + public CompletableFuture closeAsync() { + if (!manualTerminationFutureCompletion) { + terminationFuture.complete(null); + } + + return terminationFuture; + } + + @Override + public boolean isInitializedAndRunning() { + return isInitialized && !terminationFuture.isDone(); + } + + @Override + public CompletableFuture getJobMasterGatewayFuture() { + return jobMasterGatewayFuture; + } + + @Override + public CompletableFuture getResultFuture() { + return jobManagerRunnerResultFuture; + } + + @Override + public CompletableFuture getLeaderAddressFuture() { + return leaderAddressFuture; + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static final class Builder { + private CompletableFuture jobMasterGatewayFuture = + CompletableFuture.completedFuture(new TestingJobMasterGatewayBuilder().build()); + private CompletableFuture jobManagerRunnerResultFuture = + new CompletableFuture<>(); + private CompletableFuture leaderAddressFuture = + CompletableFuture.completedFuture("foobar"); + private boolean isInitialized = true; + @Nullable private CompletableFuture terminationFuture = new CompletableFuture<>(); + private boolean manualTerminationFutureCompletion = false; + + public Builder setJobMasterGatewayFuture( + CompletableFuture jobMasterGatewayFuture) { + this.jobMasterGatewayFuture = jobMasterGatewayFuture; + return this; + } + + public Builder setJobManagerRunnerResultFuture( + CompletableFuture jobManagerRunnerResultFuture) { + this.jobManagerRunnerResultFuture = jobManagerRunnerResultFuture; + return this; + } + + public Builder setLeaderAddressFuture(CompletableFuture leaderAddressFuture) { + this.leaderAddressFuture = leaderAddressFuture; + return this; + } + + public Builder setIsInitialized(boolean isInitialized) { + this.isInitialized = isInitialized; + return this; + } + + public Builder setTerminationFuture(@Nullable CompletableFuture terminationFuture) { + this.terminationFuture = terminationFuture; + return this; + } + + public Builder withManualTerminationFutureCompletion() { + this.manualTerminationFutureCompletion = true; + return this; + } + + public TestingJobMasterServiceProcess build() { + return new TestingJobMasterServiceProcess( + jobMasterGatewayFuture, + jobManagerRunnerResultFuture, + leaderAddressFuture, + isInitialized, + terminationFuture, + manualTerminationFutureCompletion); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceFactory.java index 77a4460cc17d8..a690cebf236d7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceFactory.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -18,39 +18,30 @@ package org.apache.flink.runtime.jobmaster.factories; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; -import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterService; import org.apache.flink.runtime.jobmaster.TestingJobMasterService; -import org.apache.flink.util.function.SupplierWithException; -/** - * Testing implementation of the {@link JobMasterServiceFactory} which returns a {@link JobMaster} - * mock. - */ -public class TestingJobMasterServiceFactory implements JobMasterServiceFactory { +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; - private final SupplierWithException jobMasterServiceSupplier; +/** Testing implementation of the {@link JobMasterServiceFactory}. */ +public class TestingJobMasterServiceFactory implements JobMasterServiceFactory { + private final Supplier> jobMasterServiceSupplier; public TestingJobMasterServiceFactory( - SupplierWithException jobMasterServiceSupplier) { + Supplier> jobMasterServiceSupplier) { this.jobMasterServiceSupplier = jobMasterServiceSupplier; } public TestingJobMasterServiceFactory() { - this(TestingJobMasterService::new); + this(() -> CompletableFuture.completedFuture(new TestingJobMasterService())); } @Override - public JobMasterService createJobMasterService( - JobGraph jobGraph, - JobMasterId jobMasterId, - OnCompletionActions jobCompletionActions, - ClassLoader userCodeClassloader, - long initializationTimestamp) - throws Exception { + public CompletableFuture createJobMasterService( + UUID leaderSessionId, OnCompletionActions onCompletionActions) { return jobMasterServiceSupplier.get(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java new file mode 100644 index 0000000000000..e41a095d036a7 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.factories; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobMasterServiceProcess; +import org.apache.flink.runtime.jobmaster.TestingJobMasterServiceProcess; + +import javax.annotation.Nullable; + +import java.util.UUID; +import java.util.function.Function; + +/** Testing factory for {@link JobMasterServiceProcessFactory}. */ +public class TestingJobMasterServiceProcessFactory implements JobMasterServiceProcessFactory { + + private final Function jobMasterServiceProcessFunction; + + private final JobID jobId; + + private final String jobName; + + private final long initializationTimestamp; + + private TestingJobMasterServiceProcessFactory( + Function jobMasterServiceProcessFunction, + JobID jobId, + String jobName, + long initializationTimestamp) { + this.jobMasterServiceProcessFunction = jobMasterServiceProcessFunction; + this.jobId = jobId; + this.jobName = jobName; + this.initializationTimestamp = initializationTimestamp; + } + + @Override + public JobMasterServiceProcess create(UUID leaderSessionId) { + return jobMasterServiceProcessFunction.apply(leaderSessionId); + } + + @Override + public JobID getJobId() { + return jobId; + } + + @Override + public ArchivedExecutionGraph createArchivedExecutionGraph( + JobStatus jobStatus, @Nullable Throwable cause) { + return ArchivedExecutionGraph.createFromInitializingJob( + jobId, jobName, jobStatus, cause, initializationTimestamp); + } + + public static Builder newBuilder() { + return new Builder(); + } + + public static final class Builder { + private Function jobMasterServiceProcessFunction = + ignored -> TestingJobMasterServiceProcess.newBuilder().build(); + private JobID jobId = new JobID(); + private String jobName = "foobar"; + private long initializationTimestamp = 1337L; + + public Builder setJobMasterServiceProcessFunction( + Function jobMasterServiceProcessFunction) { + this.jobMasterServiceProcessFunction = jobMasterServiceProcessFunction; + return this; + } + + public Builder setJobId(JobID jobId) { + this.jobId = jobId; + return this; + } + + public Builder setJobName(String jobName) { + this.jobName = jobName; + return this; + } + + public Builder setInitializationTimestamp(long initializationTimestamp) { + this.initializationTimestamp = initializationTimestamp; + return this; + } + + public TestingJobMasterServiceProcessFactory build() { + return new TestingJobMasterServiceProcessFactory( + jobMasterServiceProcessFunction, jobId, jobName, initializationTimestamp); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java new file mode 100644 index 0000000000000..e5479e06e2917 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster.factories; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmaster.DefaultJobMasterServiceProcess; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterService; +import org.apache.flink.runtime.jobmaster.JobMasterServiceProcess; +import org.apache.flink.runtime.jobmaster.TestingJobMasterService; + +import javax.annotation.Nullable; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +/** + * Testing implementation of the {@link JobMasterServiceProcessFactory} which returns a {@link + * JobMaster} mock. + */ +public class TestingJobMasterServiceProcessFactoryOld implements JobMasterServiceProcessFactory { + + private final CompletableFuture jobMasterServiceFuture; + private final JobID jobId; + private final JobMasterServiceFactory jobMasterServiceFactoryNg; + + public TestingJobMasterServiceProcessFactoryOld( + JobID jobId, CompletableFuture jobMasterServiceFuture) { + this.jobId = jobId; + this.jobMasterServiceFuture = jobMasterServiceFuture; + this.jobMasterServiceFactoryNg = + new TestingFutureJobMasterServiceFactory(jobMasterServiceFuture); + } + + public TestingJobMasterServiceProcessFactoryOld(JobID jobId) { + this(jobId, CompletableFuture.completedFuture(new TestingJobMasterService())); + } + + @Override + public JobMasterServiceProcess create(UUID leaderSessionID) { + return new DefaultJobMasterServiceProcess( + this.jobId, + leaderSessionID, + jobMasterServiceFactoryNg, + cause -> createArchivedExecutionGraph(JobStatus.FAILED, cause)); + } + + @Override + public JobID getJobId() { + return this.jobId; + } + + @Override + public ArchivedExecutionGraph createArchivedExecutionGraph( + JobStatus jobStatus, @Nullable Throwable cause) { + return ArchivedExecutionGraph.createFromInitializingJob( + jobId, "test-job", jobStatus, cause, System.currentTimeMillis()); + } + + public static class TestingFutureJobMasterServiceFactory implements JobMasterServiceFactory { + + final CompletableFuture jobMasterServiceFuture; + + public TestingFutureJobMasterServiceFactory( + CompletableFuture jobMasterServiceFuture) { + this.jobMasterServiceFuture = jobMasterServiceFuture; + } + + @Override + public CompletableFuture createJobMasterService( + UUID leaderSessionId, OnCompletionActions onCompletionActions) { + return jobMasterServiceFuture; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java index 4480b2b076597..6a2b23cd3963b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java @@ -677,12 +677,10 @@ public void initializeOnMaster(ClassLoader loader) { try { jobResultFuture.get(); } catch (ExecutionException e) { - assertTrue(findThrowable(e, OutOfMemoryError.class).isPresent()); + assertThat(e, FlinkMatchers.containsCause(OutOfMemoryError.class)); assertThat( - findThrowable(e, OutOfMemoryError.class) - .map(OutOfMemoryError::getMessage) - .get(), - startsWith( + e, + FlinkMatchers.containsMessage( "Java heap space. A heap space-related out-of-memory error has occurred.")); } } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterITCase.java new file mode 100644 index 0000000000000..d377fbcd79998 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterITCase.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmaster; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** Integration tests for the {@link JobMaster}. */ +public class JobMasterITCase extends TestLogger { + + @Test + public void testRejectionOfEmptyJobGraphs() throws Exception { + MiniCluster miniCluster = + new MiniCluster( + new MiniClusterConfiguration.Builder() + .setNumTaskManagers(1) + .setNumSlotsPerTaskManager(1) + .build()); + miniCluster.start(); + JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + + try { + miniCluster.submitJob(jobGraph).get(); + fail("Expect failure"); + } catch (Throwable t) { + assertThat(t, containsMessage("The given job is empty")); + } + miniCluster.close(); + } + + /** + * This test is to guard against the issue reported in FLINK-22001, where any exception from the + * JobManager initialization was not forwarded to the user. + * + *

TODO: This test relies on an internal error. Replace it with a more robust approach. + */ + @Test + public void testJobManagerInitializationExceptionsAreForwardedToTheUser() { + // we must use the LocalStreamEnvironment to reproduce this issue. + // It passes with the TestStreamEnvironment (which is initialized by the + // MiniClusterResource). The LocalStreamEnvironment is polling the JobManager for the job + // status, while TestStreamEnvironment is waiting on the resultFuture. + StreamExecutionEnvironment see = StreamExecutionEnvironment.createLocalEnvironment(); + + Source mySource = new FailOnInitializationSource(); + DataStream stream = + see.fromSource(mySource, WatermarkStrategy.noWatermarks(), "MySourceName"); + stream.addSink(new DiscardingSink<>()); + + try { + see.execute(); + } catch (Exception e) { + assertThat(e, containsMessage("Context was not yet initialized")); + } + } + + private static class FailOnInitializationSource implements Source { + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) throws Exception { + throw new RuntimeException(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, Void checkpoint) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + throw new RuntimeException(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + throw new UnsupportedOperationException(); + } + } + + private static class MockSplit implements SourceSplit { + @Override + public String splitId() { + throw new UnsupportedOperationException(); + } + } +}