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

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

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

    https://github.com/apache/flink/pull/6203#discussion_r199197307
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandlerTest.java
 ---
    @@ -47,8 +65,30 @@
      */
     public class JobSubmitHandlerTest extends TestLogger {
     
    +   @ClassRule
    +   public static final TemporaryFolder TEMPORARY_FOLDER = new 
TemporaryFolder();
    +   private static BlobServer blobServer;
    +
    +   @BeforeClass
    +   public static void setup() throws IOException {
    +           Configuration config = new Configuration();
    +           config.setString(BlobServerOptions.STORAGE_DIRECTORY,
    +                   TEMPORARY_FOLDER.newFolder().getAbsolutePath());
    +
    +           blobServer = new BlobServer(config, new VoidBlobStore());
    +           blobServer.start();
    +   }
    +
    +   @AfterClass
    +   public static void teardown() throws IOException {
    +           if (blobServer != null) {
    +                   blobServer.close();
    +           }
    +   }
    +
        @Test
        public void testSerializationFailureHandling() throws Exception {
    +           final Path jobGraphFile = TEMPORARY_FOLDER.newFile().toPath();
                DispatcherGateway mockGateway = mock(DispatcherGateway.class);
                when(mockGateway.submitJob(any(JobGraph.class), 
any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get()));
                GatewayRetriever<DispatcherGateway> mockGatewayRetriever = 
mock(GatewayRetriever.class);
    --- End diff --
    
    Here we could use instead `() -> new CompletableFuture()`


> 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)

Reply via email to