/** * Suspending job, all the running tasks will be cancelled, and communication with other components * will be disposed. * * <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by * calling the {@link #start(JobMasterId, Time)} method once we take the leadership back again. * * @param cause The reason of why this job been suspended. */ private Acknowledge suspendExecution(final Exception cause) { validateRunsInMainThread(); if (getFencingToken() == null) { log.debug("Job has already been suspended or shutdown."); return Acknowledge.get(); } // not leader anymore --> set the JobMasterId to null setFencingToken(null); try { resourceManagerLeaderRetriever.stop(); } catch (Throwable t) { log.warn("Failed to stop resource manager leader retriever when suspending.", t); } suspendAndClearExecutionGraphFields(cause); // the slot pool stops receiving messages and clears its pooled slots slotPoolGateway.suspend(); // disconnect from resource manager: closeResourceManagerConnection(cause); return Acknowledge.get(); }
private void startJobMasterServices() throws Exception { // start the slot pool make sure the slot pool now accepts messages for this leader slotPool.start(getFencingToken(), getAddress()); //TODO: Remove once the ZooKeeperLeaderRetrieval returns the stored address upon start // try to reconnect to previously known leader reconnectToResourceManager(new FlinkException("Starting JobMaster component.")); // job is ready to go, try to establish connection with resource manager // - activate leader retrieval for the resource manager // - on notification of the leader, the connection will be established and // the slot pool will start requesting slots resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); }
private void resetAndScheduleExecutionGraph() throws Exception { validateRunsInMainThread(); final CompletableFuture<Void> executionGraphAssignedFuture; if (executionGraph.getState() == JobStatus.CREATED) { executionGraphAssignedFuture = CompletableFuture.completedFuture(null); } else { suspendAndClearExecutionGraphFields(new FlinkException("ExecutionGraph is being reset in order to be rescheduled.")); final JobManagerJobMetricGroup newJobManagerJobMetricGroup = jobMetricGroupFactory.create(jobGraph); final ExecutionGraph newExecutionGraph = createAndRestoreExecutionGraph(newJobManagerJobMetricGroup); executionGraphAssignedFuture = executionGraph.getTerminationFuture().handleAsync( (JobStatus ignored, Throwable throwable) -> { assignExecutionGraph(newExecutionGraph, newJobManagerJobMetricGroup); return null; }, getMainThreadExecutor()); } executionGraphAssignedFuture.thenRun(this::scheduleExecutionGraph); }
private void reconnectToResourceManager(Exception cause) { closeResourceManagerConnection(cause); tryConnectToResourceManager(); }
/** * Suspending job, all the running tasks will be cancelled, and communication with other components * will be disposed. * * <p>Mostly job is suspended because of the leadership has been revoked, one can be restart this job by * calling the {@link #start(JobMasterId, Time)} method once we take the leadership back again. * * <p>This method is executed asynchronously * * @param cause The reason of why this job been suspended. * @param timeout for this operation * @return Future acknowledge indicating that the job has been suspended. Otherwise the future contains an exception */ public CompletableFuture<Acknowledge> suspend(final Exception cause, final Time timeout) { CompletableFuture<Acknowledge> suspendFuture = callAsyncWithoutFencing(() -> suspendExecution(cause), timeout); stop(); return suspendFuture; }
/** * Start the rpc service and begin to run the job. * * @param newJobMasterId The necessary fencing token to run the job * @param timeout for the operation * @return Future acknowledge if the job could be started. Otherwise the future contains an exception */ public CompletableFuture<Acknowledge> start(final JobMasterId newJobMasterId, final Time timeout) throws Exception { // make sure we receive RPC and async calls super.start(); return callAsyncWithoutFencing(() -> startJobExecution(newJobMasterId), timeout); }
rescaleJobGraph(operators, newParallelism, rescalingBehaviour); } catch (FlinkException e) { final String msg = String.format("Cannot rescale job %s.", jobGraph.getName()); newExecutionGraph = createExecutionGraph(newJobManagerJobMetricGroup); } catch (JobExecutionException | JobException e) { return FutureUtils.completedExceptionally( final CompletableFuture<String> savepointFuture = getJobModificationSavepoint(timeout); final CompletableFuture<ExecutionGraph> executionGraphFuture = restoreExecutionGraphFromRescalingSavepoint( newExecutionGraph, savepointFuture) getMainThreadExecutor()); suspendExecutionGraph(new FlinkException("Job is being rescaled.")); return currentExecutionGraph.getTerminationFuture(); }, getMainThreadExecutor()); clearExecutionGraphFields(); assignExecutionGraph(restoredExecutionGraph, newJobManagerJobMetricGroup); scheduleExecutionGraph(); getMainThreadExecutor());
final ExecutionGraph newExecutionGraph; try { newExecutionGraph = createAndRestoreExecutionGraph(newJobGraph, newJobManagerJobMetricGroup); } catch (Exception e) { runAsync(() -> suspendExecutionGraph(new FlinkException("Suspend the old job to update it."))); final CompletableFuture<JobStatus> terminationFuture = currentExecutionGraph.getTerminationFuture(); final CompletableFuture<Void> suspendedFuture = terminationFuture.thenAccept( clearExecutionGraphFields(); checkState(executionGraph.getState().isTerminalState()); optimizeNewExecutionGraph(newExecutionGraph, currentExecutionGraph); assignExecutionGraph(newExecutionGraph, newJobManagerJobMetricGroup); operationLogManager.clear(); scheduleExecutionGraph(); } else { final String msg = "Detected unexpected concurrent modification of ExecutionGraph."; getMainThreadExecutor()); handleJobMasterError(unexpectedError);
private Acknowledge startJobExecution(JobMasterId newJobMasterId) throws Exception { validateRunsInMainThread(); if (Objects.equals(getFencingToken(), newJobMasterId)) { log.info("Already started the job execution with JobMasterId {}.", newJobMasterId); setNewFencingToken(newJobMasterId); startJobMasterServices(); scheduleExecutionGraph(); } else { executionGraph.getReconcileFuture().thenApplyAsync( Execution execution = executionGraph.getRegisteredExecutions().get(executionAttemptId); if (execution != null) { updateTaskExecutionState(new TaskExecutionState( executionGraph.getJobID(), execution.getAttemptId(), scheduleExecutionGraph(); } else { log.error("When reconcile finished, the job is in {}, this is a logical error.", executionGraph.getState()); , getMainThreadExecutor());
private Acknowledge startJobExecution(JobMasterId newJobMasterId) throws Exception { validateRunsInMainThread(); checkNotNull(newJobMasterId, "The new JobMasterId must not be null."); if (Objects.equals(getFencingToken(), newJobMasterId)) { log.info("Already started the job execution with JobMasterId {}.", newJobMasterId); return Acknowledge.get(); } setNewFencingToken(newJobMasterId); startJobMasterServices(); log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID()); resetAndScheduleExecutionGraph(); return Acknowledge.get(); }
@Override public CompletableFuture<String> triggerSavepoint( @Nullable final String targetDirectory, final boolean cancelJob, final Time timeout) { final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); if (checkpointCoordinator == null) { return FutureUtils.completedExceptionally(new IllegalStateException( String.format("Job %s is not a streaming job.", jobGraph.getJobID()))); } if (cancelJob) { checkpointCoordinator.stopCheckpointScheduler(); } return checkpointCoordinator .triggerSavepoint(System.currentTimeMillis(), targetDirectory) .thenApply(CompletedCheckpoint::getExternalPointer) .thenApplyAsync(path -> { if (cancelJob) { log.info("Savepoint stored in {}. Now cancelling {}.", path, jobGraph.getJobID()); cancel(timeout); } return path; }, getMainThreadExecutor()) .exceptionally(throwable -> { if (cancelJob) { startCheckpointScheduler(checkpointCoordinator); } throw new CompletionException(throwable); }); }
return triggerSavepoint( null, false, () -> disposeSavepoint(savepointToDispose), scheduledExecutorService); getMainThreadExecutor());
if (savepointPath != null) { try { tryRestoreExecutionGraphFromSavepoint(newExecutionGraph, SavepointRestoreSettings.forPath(savepointPath, false)); } catch (Exception e) { final String message = String.format("Could not restore from temporary rescaling savepoint. This might indicate " + getMainThreadExecutor()) .thenRunAsync( () -> disposeSavepoint(savepointPath), scheduledExecutorService); tryRestoreExecutionGraphFromSavepoint(newExecutionGraph, jobGraph.getSavepointRestoreSettings()); } catch (Exception e) { final String message = String.format("Could not restore from initial savepoint. This might indicate " +
return CompletableFuture.completedFuture(response); } else { return getRpcService() .connect(taskManagerRpcAddress, TaskExecutorGateway.class) .handleAsync( getMainThreadExecutor());
@Override protected RetryingRegistration<ResourceManagerId, ResourceManagerGateway, JobMasterRegistrationSuccess> generateRegistration() { return new RetryingRegistration<ResourceManagerId, ResourceManagerGateway, JobMasterRegistrationSuccess>( log, getRpcService(), "ResourceManager", ResourceManagerGateway.class, getTargetAddress(), getTargetLeaderId()) { @Override protected CompletableFuture<RegistrationResponse> invokeRegistration( ResourceManagerGateway gateway, ResourceManagerId fencingToken, long timeoutMillis) { Time timeout = Time.milliseconds(timeoutMillis); return gateway.registerJobManager( jobMasterId, jobManagerResourceID, jobManagerRpcAddress, jobID, timeout); } }; }
private void connectToResourceManager() { assert(resourceManagerAddress != null); assert(resourceManagerConnection == null); assert(establishedResourceManagerConnection == null); log.info("Connecting to ResourceManager {}", resourceManagerAddress); resourceManagerConnection = new ResourceManagerConnection( log, jobGraph.getJobID(), resourceId, getAddress(), getFencingToken(), resourceManagerAddress.getAddress(), resourceManagerAddress.getResourceManagerId(), scheduledExecutorService); resourceManagerConnection.start(); }
leaderGatewayFuture.completeExceptionally(new FlinkException("JobMaster has been shut down.")); jobMaster.shutDown(); final CompletableFuture<Void> jobManagerTerminationFuture = jobMaster.getTerminationFuture();
clearExecutionGraphFields(); final ExecutionGraph newExecutionGraph = createAndRestoreExecutionGraph(jobGraph, newJobManagerJobMetricGroup); assignExecutionGraph(newExecutionGraph, newJobManagerJobMetricGroup); clearExecutionGraphFields(); final ExecutionGraph newExecutionGraph = createAndRestoreExecutionGraph(jobGraph, newJobManagerJobMetricGroup); assignExecutionGraph(newExecutionGraph, newJobManagerJobMetricGroup); return;
private void jobStatusChanged( final JobStatus newJobStatus, long timestamp, @Nullable final Throwable error) { validateRunsInMainThread(); if (newJobStatus.isGloballyTerminalState()) { final ArchivedExecutionGraph archivedExecutionGraph = ArchivedExecutionGraph.createFrom(executionGraph); scheduledExecutorService.execute(() -> jobCompletionActions.jobReachedGloballyTerminalState(archivedExecutionGraph)); } }
closeResourceManagerConnection(new Exception( "ResourceManager leader changed to new address " + resourceManagerAddress)); jobGraph.getJobID(), resourceId, getAddress(), leaderSessionID, resourceManagerAddress,