[
https://issues.apache.org/jira/browse/FLINK-9280?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16522086#comment-16522086
]
ASF GitHub Bot commented on FLINK-9280:
---------------------------------------
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/6203#discussion_r197735309
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandler.java
---
@@ -54,18 +69,89 @@ public JobSubmitHandler(
@Override
protected CompletableFuture<JobSubmitResponseBody>
handleRequest(@Nonnull HandlerRequest<JobSubmitRequestBody,
EmptyMessageParameters> request, @Nonnull DispatcherGateway gateway) throws
RestHandlerException {
- JobGraph jobGraph;
- try {
- ObjectInputStream objectIn = new ObjectInputStream(new
ByteArrayInputStream(request.getRequestBody().serializedJobGraph));
- jobGraph = (JobGraph) objectIn.readObject();
- } catch (Exception e) {
- throw new RestHandlerException(
- "Failed to deserialize JobGraph.",
- HttpResponseStatus.BAD_REQUEST,
- e);
+ Collection<Path> uploadedFiles = request.getUploadedFiles();
--- End diff --
I was wondering when you might bring that up :)
I'm not a fan of exposing `File`.
* whether it _really is a local file_ shouldn't be relevant to the handler
* nio Paths are more flexible than files; for example if the
DistributedCache were to return files we wouldn't have to extract zips as we
could mount it with a `ZipFileSystem` instead
* always having to convert using `File#toPath` is rather tedious
> 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
> Labels: pull-request-available
> 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)