Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6147#discussion_r195088315
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
 ---
    @@ -65,7 +86,96 @@ public JobSubmitHandler(
                                e);
                }
     
    -           return gateway.submitJob(jobGraph, timeout)
    +           updateJarEntriesInJobGraph(jobGraph, 
requestBody.getUploadedJars(), log);
    +           updateUserArtifactEntriesInJobGraph(jobGraph, 
requestBody.getUploadedArtifacts(), log);
    +
    +           CompletableFuture<Integer> blobServerPortFuture = 
gateway.getBlobServerPort(timeout);
    +
    +           CompletableFuture<JobGraph> jobGraphFuture = 
blobServerPortFuture.thenApply(blobServerPort -> {
    +                   final InetSocketAddress address = new 
InetSocketAddress(getDispatcherHost(gateway), blobServerPort);
    +                   final List<PermanentBlobKey> keys;
    +                   try {
    +                           keys = BlobClient.uploadFiles(address, config, 
jobGraph.getJobID(), jobGraph.getUserJars());
    +                           jobGraph.uploadUserArtifacts(address, config);
    +                   } catch (IOException ioe) {
    +                           log.error("Could not upload job jar files.", 
ioe);
    +                           throw new CompletionException(new 
RestHandlerException("Could not upload job jar files.", 
HttpResponseStatus.INTERNAL_SERVER_ERROR));
    +                   }
    +
    +                   for (PermanentBlobKey key : keys) {
    +                           jobGraph.addUserJarBlobKey(key);
    +                   }
    +
    +                   return jobGraph;
    +           });
    +
    +           CompletableFuture<JobSubmitResponseBody> submissionFuture = 
jobGraphFuture
    +                   .thenCompose(finalizedJobGraph -> 
gateway.submitJob(jobGraph, timeout))
                        .thenApply(ack -> new JobSubmitResponseBody("/jobs/" + 
jobGraph.getJobID()));
    +
    +           CompletableFuture<Void> submissionCleanupFuture = 
submissionFuture.thenRun(requestBody::cleanup);
    +
    +           return submissionFuture.thenCombine(submissionCleanupFuture, 
(responseBody, ignored) -> responseBody);
    +   }
    +
    +   /**
    +    * Updates the jar entries in the given JobGraph to refer to the 
uploaded jar files instead of client-local files.
    +    */
    +   private static void updateJarEntriesInJobGraph(JobGraph jobGraph, 
Collection<Path> uploadedJars, Logger log) {
    --- End diff --
    
    Why do we need to update the jar file names in the `JobGraph`? I thought 
`JobGraph#userJars` is only used by the client to learn which jars to upload to 
the cluster.


---

Reply via email to