/** * Executes the remote job. * * @param streamGraph * Stream Graph to execute * @param jarFiles * List of jar file URLs to ship to the cluster * @return The result of the job execution, containing elapsed time and accumulators. */ @Override protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List<URL> jarFiles) throws ProgramInvocationException { URL jarUrl; try { jarUrl = flinkILoop.writeFilesToDisk().getAbsoluteFile().toURI().toURL(); } catch (MalformedURLException e) { throw new ProgramInvocationException("Could not write the user code classes to disk.", streamGraph.getJobGraph().getJobID(), e); } List<URL> allJarFiles = new ArrayList<>(jarFiles.size() + 1); allJarFiles.addAll(jarFiles); allJarFiles.add(jarUrl); return super.executeRemotely(streamGraph, allJarFiles); }
@Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { log.info("Submitting job {} (detached: {}).", jobGraph.getJobID(), isDetached()); } catch (Exception e) { throw new ProgramInvocationException("Could not submit job", jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e)); ignored -> requestJobResult(jobGraph.getJobID())); } catch (Exception e) { throw new ProgramInvocationException("Could not retrieve the execution result.", jobGraph.getJobID(), ExceptionUtils.stripExecutionException(e)); return lastJobExecutionResult; } catch (JobExecutionException e) { throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e); } catch (IOException | ClassNotFoundException e) { throw new ProgramInvocationException("Job failed.", jobGraph.getJobID(), e);
@Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { final CompletableFuture<JobSubmissionResult> jobSubmissionResultFuture = submitJob(jobGraph); if (isDetached()) { try { return jobSubmissionResultFuture.get(); } catch (InterruptedException | ExecutionException e) { ExceptionUtils.checkInterrupted(e); throw new ProgramInvocationException("Could not run job in detached mode.", jobGraph.getJobID(), e); } } else { final CompletableFuture<JobResult> jobResultFuture = jobSubmissionResultFuture.thenCompose( (JobSubmissionResult ignored) -> requestJobResult(jobGraph.getJobID())); final JobResult jobResult; try { jobResult = jobResultFuture.get(); } catch (InterruptedException | ExecutionException e) { ExceptionUtils.checkInterrupted(e); throw new ProgramInvocationException("Could not run job", jobGraph.getJobID(), e); } try { return jobResult.toJobExecutionResult(classLoader); } catch (JobExecutionException e) { throw new ProgramInvocationException("Job failed", jobGraph.getJobID(), e); } catch (IOException | ClassNotFoundException e) { throw new ProgramInvocationException("Job failed", jobGraph.getJobID(), e); } } }
private String cancelWithSavepoint() throws Exception { return clusterClient.cancelWithSavepoint( jobGraph.getJobID(), savepointDirectory.toAbsolutePath().toString()); }
private static String runJobAndGetExternalizedCheckpoint(StateBackend backend, File checkpointDir, @Nullable String externalCheckpoint, ClusterClient<?> client) throws Exception { JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); client.submitJob(initialJobGraph, ResumeCheckpointManuallyITCase.class.getClassLoader()); // wait until all sources have been started NotifyingInfiniteTupleSource.countDownLatch.await(); waitUntilExternalizedCheckpointCreated(checkpointDir, initialJobGraph.getJobID()); client.cancel(initialJobGraph.getJobID()); waitUntilCanceled(initialJobGraph.getJobID(), client); return getExternalizedCheckpointCheckpointPath(checkpointDir, initialJobGraph.getJobID()); }
@Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { if (isDetached()) { if (newlyCreatedCluster) { stopAfterJob(jobGraph.getJobID()); } return super.runDetached(jobGraph, classLoader); } else { return super.run(jobGraph, classLoader); } }
private void waitForJob() throws Exception { for (int i = 0; i < 60; i++) { try { final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); assertThat(jobStatus.isGloballyTerminalState(), equalTo(false)); if (jobStatus == JobStatus.RUNNING) { return; } } catch (ExecutionException ignored) { // JobManagerRunner is not yet registered in Dispatcher } Thread.sleep(1000); } throw new AssertionError("Job did not become running within timeout."); }
@Test public void testStopJobAfterSavepoint() throws Exception { setUpWithCheckpointInterval(10L); final String savepointLocation = cancelWithSavepoint(); final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(); assertThat(jobStatus, isOneOf(JobStatus.CANCELED, JobStatus.CANCELLING)); final List<Path> savepoints; try (Stream<Path> savepointFiles = Files.list(savepointDirectory)) { savepoints = savepointFiles.map(Path::getFileName).collect(Collectors.toList()); } assertThat(savepoints, hasItem(Paths.get(savepointLocation).getFileName())); }
@Test public void testStopJobAfterSavepointWithDeactivatedPeriodicCheckpointing() throws Exception { // set checkpointInterval to Long.MAX_VALUE, which means deactivated checkpointing setUpWithCheckpointInterval(Long.MAX_VALUE); final String savepointLocation = cancelWithSavepoint(); final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); assertThat(jobStatus, isOneOf(JobStatus.CANCELED, JobStatus.CANCELLING)); final List<Path> savepoints; try (Stream<Path> savepointFiles = Files.list(savepointDirectory)) { savepoints = savepointFiles.map(Path::getFileName).collect(Collectors.toList()); } assertThat(savepoints, hasItem(Paths.get(savepointLocation).getFileName())); }
@Before public void setUp() throws Exception { restServerEndpointConfiguration = RestServerEndpointConfiguration.fromConfiguration(restConfig); mockGatewayRetriever = () -> CompletableFuture.completedFuture(mockRestfulGateway); executor = Executors.newSingleThreadExecutor(new ExecutorThreadFactory(RestClusterClientTest.class.getSimpleName())); jobGraph = new JobGraph("testjob"); jobId = jobGraph.getJobID(); }
@Test public void testDoNotCancelJobIfSavepointFails() throws Exception { setUpWithCheckpointInterval(10L); try { Files.setPosixFilePermissions(savepointDirectory, Collections.emptySet()); } catch (IOException e) { Assume.assumeNoException(e); } try { cancelWithSavepoint(); } catch (Exception e) { assertThat(ExceptionUtils.findThrowable(e, CheckpointTriggerException.class).isPresent(), equalTo(true)); } final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); assertThat(jobStatus, equalTo(JobStatus.RUNNING)); // assert that checkpoints are continued to be triggered triggerCheckpointLatch = new CountDownLatch(1); assertThat(triggerCheckpointLatch.await(60L, TimeUnit.SECONDS), equalTo(true)); }
private <T> void deployJobOnNewCluster( ClusterDescriptor<T> clusterDescriptor, JobGraph jobGraph, Result<T> result, ClassLoader classLoader) throws Exception { ClusterClient<T> clusterClient = null; try { // deploy job cluster with job attached clusterClient = clusterDescriptor.deployJobCluster(context.getClusterSpec(), jobGraph, false); // save information about the new cluster result.setClusterInformation(clusterClient.getClusterId(), clusterClient.getWebInterfaceURL()); // get result if (awaitJobResult) { // we need to hard cast for now final JobExecutionResult jobResult = ((RestClusterClient<T>) clusterClient) .requestJobResult(jobGraph.getJobID()) .get() .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails executionResultBucket.add(jobResult); } } finally { try { if (clusterClient != null) { clusterClient.shutdown(); } } catch (Exception e) { // ignore } } }
/** * Tests that cancel with savepoint without a properly configured savepoint * directory, will fail with a meaningful exception message. */ @Test public void testCancelWithSavepointWithoutConfiguredSavepointDirectory() throws Exception { setUpWithCheckpointInterval(10L); try { clusterClient.cancelWithSavepoint(jobGraph.getJobID(), null); } catch (Exception e) { if (!ExceptionUtils.findThrowableWithMessage(e, "savepoint directory").isPresent()) { throw e; } } }
public static void addUserArtifactEntries(Collection<Tuple2<String, DistributedCache.DistributedCacheEntry>> userArtifacts, JobGraph jobGraph) { if (!userArtifacts.isEmpty()) { try { java.nio.file.Path tmpDir = Files.createTempDirectory("flink-distributed-cache-" + jobGraph.getJobID()); for (Tuple2<String, DistributedCache.DistributedCacheEntry> originalEntry : userArtifacts) { Path filePath = new Path(originalEntry.f1.filePath); boolean isLocalDir = false; try { FileSystem sourceFs = filePath.getFileSystem(); isLocalDir = !sourceFs.isDistributedFS() && sourceFs.getFileStatus(filePath).isDir(); } catch (IOException ioe) { LOG.warn("Could not determine whether {} denotes a local path.", filePath, ioe); } // zip local directories because we only support file uploads DistributedCache.DistributedCacheEntry entry; if (isLocalDir) { Path zip = FileUtils.compressDirectory(filePath, new Path(tmpDir.toString(), filePath.getName() + ".zip")); entry = new DistributedCache.DistributedCacheEntry(zip.toString(), originalEntry.f1.isExecutable, true); } else { entry = new DistributedCache.DistributedCacheEntry(filePath.toString(), originalEntry.f1.isExecutable, false); } jobGraph.addUserArtifact(originalEntry.f0, entry); } } catch (IOException ioe) { throw new FlinkRuntimeException("Could not compress distributed-cache artifacts.", ioe); } } }
private void restoreJob(ClassLoader classLoader, ClusterClient<?> clusterClient, Deadline deadline, String savepointPath) throws Exception { JobGraph jobToRestore = createJobGraph(ExecutionMode.RESTORE); jobToRestore.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, allowNonRestoredState)); assertNotNull("Job doesn't have a JobID.", jobToRestore.getJobID()); clusterClient.submitJob(jobToRestore, classLoader); CompletableFuture<JobStatus> jobStatusFuture = FutureUtils.retrySuccessfulWithDelay( () -> clusterClient.getJobStatus(jobToRestore.getJobID()), Time.milliseconds(50), deadline, (jobStatus) -> jobStatus == JobStatus.FINISHED, TestingUtils.defaultScheduledExecutor()); assertEquals( JobStatus.FINISHED, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); }
AutoCancellableJob(Deadline deadline, final ClusterClient<?> clusterClient, final StreamExecutionEnvironment env) { Preconditions.checkNotNull(env); this.clusterClient = Preconditions.checkNotNull(clusterClient); this.jobGraph = env.getStreamGraph().getJobGraph(); this.jobId = Preconditions.checkNotNull(jobGraph.getJobID()); this.deadline = deadline; }
private String submitJobAndTakeSavepoint(MiniClusterResourceFactory clusterFactory, int parallelism) throws Exception { final JobGraph jobGraph = createJobGraph(parallelism, 0, 1000); final JobID jobId = jobGraph.getJobID(); StatefulCounter.resetForTest(parallelism); MiniClusterWithClientResource cluster = clusterFactory.get(); cluster.before(); ClusterClient<?> client = cluster.getClusterClient(); try { client.setDetached(true); client.submitJob(jobGraph, SavepointITCase.class.getClassLoader()); StatefulCounter.getProgressLatch().await(); return client.triggerSavepoint(jobId, null).get(); } finally { cluster.after(); StatefulCounter.resetForTest(parallelism); } }
private <C> ProgramTargetDescriptor executeUpdateInternal(ExecutionContext<C> context, String statement) { final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); applyUpdate(context, envInst.getTableEnvironment(), envInst.getQueryConfig(), statement); // create job graph with dependencies final String jobName = context.getSessionContext().getName() + ": " + statement; final JobGraph jobGraph; try { jobGraph = envInst.createJobGraph(jobName); } catch (Throwable t) { // catch everything such that the statement does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); } // create execution final BasicResult<C> result = new BasicResult<>(); final ProgramDeployer<C> deployer = new ProgramDeployer<>( context, jobName, jobGraph, result, false); // blocking deployment deployer.run(); return ProgramTargetDescriptor.of( result.getClusterId(), jobGraph.getJobID(), result.getWebInterfaceUrl()); }
@Override public void handleMessage(Object message) { if (message instanceof JobManagerMessages.SubmitJob) { JobID jid = ((JobManagerMessages.SubmitJob) message).jobGraph().getJobID(); getSender().tell( decorateMessage(new JobManagerMessages.JobSubmitSuccess(jid)), getSelf()); } else if (message.getClass() == JobManagerMessages.getRequestLeaderSessionID().getClass()) { getSender().tell( decorateMessage(new JobManagerMessages.ResponseLeaderSessionID(leaderSessionID)), getSelf()); } else if (message instanceof JobManagerMessages.RequestBlobManagerPort$) { getSender().tell(1337, getSelf()); } else { getSender().tell( decorateMessage(new Status.Failure(new Exception("Unknown message " + message))), getSelf()); } }