/** * Tells the JobManager to finish the sessions (jobs) defined by the given IDs. * * @param jobIds The IDs that identify the sessions. */ public void endSessions(List<JobID> jobIds) throws Exception { if (jobIds == null) { throw new IllegalArgumentException("The JobIDs must not be null"); } ActorGateway jobManagerGateway = getJobManagerGateway(); for (JobID jid : jobIds) { if (jid != null) { log.info("Telling job manager to end the session {}.", jid); jobManagerGateway.tell(new JobManagerMessages.RemoveCachedJob(jid)); } } }
@Override public GetClusterStatusResponse getClusterStatus() { ActorGateway jmGateway; try { jmGateway = getJobManagerGateway(); Future<Object> future = jmGateway.ask(GetClusterStatus.getInstance(), timeout); Object result = Await.result(future, timeout); if (result instanceof GetClusterStatusResponse) { return (GetClusterStatusResponse) result; } else { throw new RuntimeException("Received the wrong reply " + result + " from cluster."); } } catch (Exception e) { throw new RuntimeException("Couldn't retrieve the cluster status.", e); } }
ActorGateway jm = cluster.getLeaderGateway(deadline.timeLeft()); cluster.waitForTaskManagersToBeRegisteredAtJobManager(jm.actor()); log.info("Sent recover all jobs manually to job manager {}.", jm.path()); jm.tell(JobManagerMessages.getRecoverAllJobs()); Future<Object> future = jm.ask(new WaitForAllVerticesToBeRunningOrFinished(graph.getJobID()), deadline.timeLeft()); log.info("Kill job manager {}.", jm.path()); jm.tell(TestingJobManagerMessages.getDisablePostStop()); jm.tell(Kill.getInstance());
ActorGateway leader = new AkkaActorGateway(leaderRef, leaderId); leader.tell(new SubmitJob(jobGraph, ListeningBehaviour.DETACHED));
resourceManagerFuture.thenCombine(leaderGatewayFuture, (resourceManagerGateway, leaderGateway) -> { resourceManagerGateway.tell( new NotifyResourceStarted(YarnFlinkResourceManager.extractResourceID(container)), leaderGateway); final AkkaActorGateway leader1Gateway = new AkkaActorGateway(leader1, leaderSessionID); final AkkaActorGateway resourceManagerGateway = new AkkaActorGateway(resourceManager, leaderSessionID); resourceManagerGateway.tell(new RegisterResourceManagerSuccessful(leader1, Collections.emptyList())); Future<Object> taskManagerRegisteredFuture = resourceManagerGateway.ask(new NotifyWhenResourcesRegistered(numInitialTaskManagers), deadline.timeLeft()); resourceManagerGateway.tell(new RegisterResourceManagerSuccessful(leader1, Collections.emptyList())); resourceManagerGateway.tell( new NotifyResourceStarted(YarnFlinkResourceManager.extractResourceID(container)), leader1Gateway); Future<Object> numberOfRegisteredResourcesFuture = resourceManagerGateway.ask(RequestNumberOfRegisteredResources.INSTANCE, deadline.timeLeft());
assertEquals(expectedAddress, gateway.path()); assertEquals(leaderId, gateway.leaderSessionID()); } finally { if (actorRef != null) {
(double) taskManagerInfo.getHardwareDescription().getSizeOfJvmHeap() / (double) expectedHeadSize, is(closeTo(1.0, 0.15))); } finally {
/** * This method is only available if the cluster hasn't been started in detached mode. */ @Override public GetClusterStatusResponse getClusterStatus() { try { final Future<Object> clusterStatusOption = getJobManagerGateway().ask( GetClusterStatus.getInstance(), akkaDuration); return (GetClusterStatusResponse) Await.result(clusterStatusOption, akkaDuration); } catch (Exception e) { throw new RuntimeException("Unable to get ClusterClient status from Application Client", e); } }
jmGateway.tell(PoisonPill.getInstance());
private void stopAfterJob(JobID jobID) { Preconditions.checkNotNull(jobID, "The job id must not be null"); try { Future<Object> replyFuture = getJobManagerGateway().ask( new ShutdownClusterAfterJob(jobID), akkaDuration); Await.ready(replyFuture, akkaDuration); } catch (Exception e) { throw new RuntimeException("Unable to tell application master to stop once the specified job has been finised", e); } }
/** * Cancels a job identified by the job id. * @param jobId the job id * @throws Exception In case an error occurred. */ public void cancel(JobID jobId) throws Exception { final ActorGateway jobManager = getJobManagerGateway(); Object cancelMsg = new JobManagerMessages.CancelJob(jobId); Future<Object> response = jobManager.ask(cancelMsg, timeout); final Object rc = Await.result(response, timeout); if (rc instanceof JobManagerMessages.CancellationSuccess) { // no further action required } else if (rc instanceof JobManagerMessages.CancellationFailure) { throw new Exception("Canceling the job with ID " + jobId + " failed.", ((JobManagerMessages.CancellationFailure) rc).cause()); } else { throw new IllegalStateException("Unexpected response: " + rc); } }
/** * Stops a program on Flink cluster whose job-manager is configured in this client's configuration. * Stopping works only for streaming programs. Be aware, that the program might continue to run for * a while after sending the stop command, because after sources stopped to emit data all operators * need to finish processing. * * @param jobId * the job ID of the streaming program to stop * @throws Exception * If the job ID is invalid (ie, is unknown or refers to a batch job) or if sending the stop signal * failed. That might be due to an I/O problem, ie, the job-manager is unreachable. */ public void stop(final JobID jobId) throws Exception { final ActorGateway jobManager = getJobManagerGateway(); Future<Object> response = jobManager.ask(new JobManagerMessages.StopJob(jobId), timeout); final Object rc = Await.result(response, timeout); if (rc instanceof JobManagerMessages.StoppingSuccess) { // no further action required } else if (rc instanceof JobManagerMessages.StoppingFailure) { throw new Exception("Stopping the job with ID " + jobId + " failed.", ((JobManagerMessages.StoppingFailure) rc).cause()); } else { throw new IllegalStateException("Unexpected response: " + rc); } }
/** * Requests the {@link JobStatus} of the job with the given {@link JobID}. */ public CompletableFuture<JobStatus> getJobStatus(JobID jobId) { final ActorGateway jobManager; try { jobManager = getJobManagerGateway(); } catch (FlinkException e) { throw new RuntimeException("Could not retrieve JobManage gateway.", e); } Future<Object> response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); CompletableFuture<Object> javaFuture = FutureUtils.toJava(response); return javaFuture.thenApply((responseMessage) -> { if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { return ((JobManagerMessages.CurrentJobStatus) responseMessage).status(); } else if (responseMessage instanceof JobManagerMessages.JobNotFound) { throw new CompletionException( new IllegalStateException("Could not find job with JobId " + jobId)); } else { throw new CompletionException( new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); } }); }
/** * Cancels a job identified by the job id and triggers a savepoint. * @param jobId the job id * @param savepointDirectory directory the savepoint should be written to * @return path where the savepoint is located * @throws Exception In case an error occurred. */ public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { final ActorGateway jobManager = getJobManagerGateway(); Object cancelMsg = new JobManagerMessages.CancelJobWithSavepoint(jobId, savepointDirectory); Future<Object> response = jobManager.ask(cancelMsg, timeout); final Object rc = Await.result(response, timeout); if (rc instanceof JobManagerMessages.CancellationSuccess) { JobManagerMessages.CancellationSuccess success = (JobManagerMessages.CancellationSuccess) rc; return success.savepointPath(); } else if (rc instanceof JobManagerMessages.CancellationFailure) { throw new Exception("Cancel & savepoint for the job with ID " + jobId + " failed.", ((JobManagerMessages.CancellationFailure) rc).cause()); } else { throw new IllegalStateException("Unexpected response: " + rc); } }
/** * Requests and returns the accumulators for the given job identifier. Accumulators can be * requested while a is running or after it has finished. * @param jobID The job identifier of a job. * @param loader The class loader for deserializing the accumulator results. * @return A Map containing the accumulator's name and its value. */ public Map<String, OptionalFailure<Object>> getAccumulators(JobID jobID, ClassLoader loader) throws Exception { ActorGateway jobManagerGateway = getJobManagerGateway(); Future<Object> response; try { response = jobManagerGateway.ask(new RequestAccumulatorResults(jobID), timeout); } catch (Exception e) { throw new Exception("Failed to query the job manager gateway for accumulators.", e); } Object result = Await.result(response, timeout); if (result instanceof AccumulatorResultsFound) { Map<String, SerializedValue<OptionalFailure<Object>>> serializedAccumulators = ((AccumulatorResultsFound) result).result(); return AccumulatorHelper.deserializeAccumulators(serializedAccumulators, loader); } else if (result instanceof AccumulatorResultsErroneous) { throw ((AccumulatorResultsErroneous) result).cause(); } else { throw new Exception("Failed to fetch accumulators for the job " + jobID + "."); } }
/** * Lists the currently running and finished jobs on the cluster. * * @return future collection of running and finished jobs * @throws Exception if no connection to the cluster could be established */ public CompletableFuture<Collection<JobStatusMessage>> listJobs() throws Exception { final ActorGateway jobManager = getJobManagerGateway(); Future<Object> response = jobManager.ask(new RequestJobDetails(true, false), timeout); CompletableFuture<Object> responseFuture = FutureUtils.toJava(response); return responseFuture.thenApply((responseMessage) -> { if (responseMessage instanceof MultipleJobsDetails) { MultipleJobsDetails details = (MultipleJobsDetails) responseMessage; final Collection<JobDetails> jobDetails = details.getJobs(); Collection<JobStatusMessage> flattenedDetails = new ArrayList<>(jobDetails.size()); jobDetails.forEach(detail -> flattenedDetails.add(new JobStatusMessage(detail.getJobId(), detail.getJobName(), detail.getStatus(), detail.getStartTime()))); return flattenedDetails; } else { throw new CompletionException( new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); } }); }
/** * Triggers a savepoint for the job identified by the job id. The savepoint will be written to the given savepoint * directory, or {@link org.apache.flink.configuration.CheckpointingOptions#SAVEPOINT_DIRECTORY} if it is null. * * @param jobId job id * @param savepointDirectory directory the savepoint should be written to * @return path future where the savepoint is located * @throws FlinkException if no connection to the cluster could be established */ public CompletableFuture<String> triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException { final ActorGateway jobManager = getJobManagerGateway(); Future<Object> response = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobId, Option.<String>apply(savepointDirectory)), new FiniteDuration(1, TimeUnit.HOURS)); CompletableFuture<Object> responseFuture = FutureUtils.toJava(response); return responseFuture.thenApply((responseMessage) -> { if (responseMessage instanceof JobManagerMessages.TriggerSavepointSuccess) { JobManagerMessages.TriggerSavepointSuccess success = (JobManagerMessages.TriggerSavepointSuccess) responseMessage; return success.savepointPath(); } else if (responseMessage instanceof JobManagerMessages.TriggerSavepointFailure) { JobManagerMessages.TriggerSavepointFailure failure = (JobManagerMessages.TriggerSavepointFailure) responseMessage; throw new CompletionException(failure.cause()); } else { throw new CompletionException( new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); } }); }
public CompletableFuture<Acknowledge> disposeSavepoint(String savepointPath) throws FlinkException { final ActorGateway jobManager = getJobManagerGateway(); Object msg = new JobManagerMessages.DisposeSavepoint(savepointPath); CompletableFuture<Object> responseFuture = FutureUtils.toJava( jobManager.ask( msg, timeout)); return responseFuture.thenApply( (Object response) -> { if (response instanceof JobManagerMessages.DisposeSavepointSuccess$) { return Acknowledge.get(); } else if (response instanceof JobManagerMessages.DisposeSavepointFailure) { JobManagerMessages.DisposeSavepointFailure failureResponse = (JobManagerMessages.DisposeSavepointFailure) response; if (failureResponse.cause() instanceof ClassNotFoundException) { throw new CompletionException( new ClassNotFoundException("Savepoint disposal failed, because of a " + "missing class. This is most likely caused by a custom state " + "instance, which cannot be disposed without the user code class " + "loader. Please provide the program jar with which you have created " + "the savepoint via -j <JAR> for disposal.", failureResponse.cause().getCause())); } else { throw new CompletionException(failureResponse.cause()); } } else { throw new CompletionException(new FlinkRuntimeException("Unknown response type " + response.getClass().getSimpleName() + '.')); } }); }
private void stopAfterJob(ClusterClient client, JobID jobID) { Preconditions.checkNotNull(jobID, "The job id must not be null"); try { Future<Object> replyFuture = client.getJobManagerGateway().ask( new ShutdownClusterAfterJob(jobID), AKKA_TIMEOUT); Await.ready(replyFuture, AKKA_TIMEOUT); } catch (Exception e) { throw new RuntimeException("Unable to tell application master to stop" + " once the specified job has been finished", e); } } }