[ 
https://issues.apache.org/jira/browse/FLINK-9280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16511248#comment-16511248
 ] 

ASF GitHub Bot commented on FLINK-9280:
---------------------------------------

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

    https://github.com/apache/flink/pull/6147#discussion_r195111137
  
    --- 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 --
    
    correct, this field is used in `JobGraph#uploadUserJars` to upload the jars 
to the blob-server. Since this is now done on the server but the original 
entries still point to client-local files we have to update the entries.
    
    If we move the upload out of the jobgraph we can skip this step.


> Extend JobSubmitHandler to accept jar files
> -------------------------------------------
>
>                 Key: FLINK-9280
>                 URL: https://issues.apache.org/jira/browse/FLINK-9280
>             Project: Flink
>          Issue Type: New Feature
>          Components: Job-Submission, REST
>    Affects Versions: 1.5.0
>            Reporter: Chesnay Schepler
>            Assignee: Chesnay Schepler
>            Priority: Critical
>             Fix For: 1.6.0, 1.5.1
>
>
> The job submission through the CLI first uploads all require jars to the blob 
> server, sets the blob keys in the jobgraph, and then uploads this graph to 
> The {{JobSubmitHandler}} which submits it to the Dispatcher.
> This process has the downside that it requires jars to be uploaded to the 
> blobserver before submitting the job graph, which does not happen via REST.
> I propose an extension to the the {{JobSubmitHandler}} to also accept an 
> optional list of jar files, that were previously uploaded through the 
> {{JarUploadHandler}}. If present, the handler would upload these jars to the 
> blobserver and set the blob keys.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to