TisonKun commented on a change in pull request #11460: 
[FLINK-16655][FLINK-16657] Introduce embedded executor and use it for Web 
Submission
URL: https://github.com/apache/flink/pull/11460#discussion_r401497836
 
 

 ##########
 File path: 
flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java
 ##########
 @@ -67,41 +69,39 @@ public JarRunHandler(
                        final MessageHeaders<JarRunRequestBody, 
JarRunResponseBody, JarRunMessageParameters> messageHeaders,
                        final Path jarDir,
                        final Configuration configuration,
-                       final Executor executor) {
+                       final Executor executor,
+                       final Supplier<ApplicationRunner> 
applicationRunnerSupplier) {
                super(leaderRetriever, timeout, responseHeaders, 
messageHeaders);
 
                this.jarDir = requireNonNull(jarDir);
                this.configuration = requireNonNull(configuration);
                this.executor = requireNonNull(executor);
+
+               this.applicationRunner = applicationRunnerSupplier.get();
        }
 
        @Override
        protected CompletableFuture<JarRunResponseBody> handleRequest(
                        @Nonnull final HandlerRequest<JarRunRequestBody, 
JarRunMessageParameters> request,
                        @Nonnull final DispatcherGateway gateway) throws 
RestHandlerException {
-               final JarHandlerContext context = 
JarHandlerContext.fromRequest(request, jarDir, log);
-
-               final SavepointRestoreSettings savepointRestoreSettings = 
getSavepointRestoreSettings(request);
-
-               final CompletableFuture<JobGraph> jobGraphFuture = 
getJobGraphAsync(context, savepointRestoreSettings);
-
-               CompletableFuture<Integer> blobServerPortFuture = 
gateway.getBlobServerPort(timeout);
 
-               CompletableFuture<JobGraph> jarUploadFuture = 
jobGraphFuture.thenCombine(blobServerPortFuture, (jobGraph, blobServerPort) -> {
-                       final InetSocketAddress address = new 
InetSocketAddress(gateway.getHostname(), blobServerPort);
-                       try {
-                               
ClientUtils.extractAndUploadJobGraphFiles(jobGraph, () -> new 
BlobClient(address, configuration));
-                       } catch (FlinkException e) {
-                               throw new CompletionException(e);
-                       }
+               configuration.set(DeploymentOptions.ATTACHED, false);
+               configuration.set(DeploymentOptions.TARGET, 
EmbeddedExecutor.NAME);
 
-                       return jobGraph;
-               });
-
-               CompletableFuture<Acknowledge> jobSubmissionFuture = 
jarUploadFuture.thenCompose(jobGraph -> gateway.submitJob(jobGraph, timeout));
-
-               return jobSubmissionFuture
-                       .thenCombine(jarUploadFuture, (ack, jobGraph) -> new 
JarRunResponseBody(jobGraph.getJobID()));
+               final JarHandlerContext context = 
JarHandlerContext.fromRequest(request, jarDir, log);
+               context.applyOnConfiguration(configuration);
+               
SavepointRestoreSettings.toConfiguration(getSavepointRestoreSettings(request), 
configuration);
+
+               final PackagedProgram program = 
context.toPackagedProgram(configuration);
+
+               return CompletableFuture
+                               .supplyAsync(() -> 
applicationRunner.run(gateway, program, configuration), executor)
+                               .thenApply(jobIds -> {
+                                       if (jobIds.isEmpty()) {
+                                               throw new 
CompletionException(new ProgramInvocationException("No jobs submitted."));
+                                       }
+                                       return new 
JarRunResponseBody(jobIds.get(0));
 
 Review comment:
   It is kind of subtle that we possibly actually have multiple job executed 
but return id of the first one. Maybe we should mention this manner somewhere.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to