[ https://issues.apache.org/jira/browse/FLINK-9211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16458404#comment-16458404 ]
ASF GitHub Bot commented on FLINK-9211: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/5903#discussion_r184936545 --- Diff: flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java --- @@ -105,9 +110,32 @@ public JarRunHandler( savepointRestoreSettings, parallelism); - return jobGraphFuture.thenCompose(jobGraph -> restClusterClient - .submitJob(jobGraph) - .thenApply((jobSubmitResponseBody -> new JarRunResponseBody(jobGraph.getJobID())))) + CompletableFuture<Integer> blobServerPortFuture = gateway.getBlobServerPort(timeout); + + CompletableFuture<JobGraph> jarUploadFuture = jobGraphFuture.thenCombine(blobServerPortFuture, (jobGraph, blobServerPort) -> { + final InetSocketAddress address = new InetSocketAddress(getDispatcherHost(gateway), blobServerPort); + final List<PermanentBlobKey> keys; + try { + keys = BlobClient.uploadJarFiles(address, configuration, jobGraph.getJobID(), jobGraph.getUserJars()); + } catch (IOException ioe) { + throw new CompletionException(new FlinkException("Could not upload job jar files.", ioe)); + } + + for (PermanentBlobKey key : keys) { + jobGraph.addBlob(key); + } + + return jobGraph; + }); + + CompletableFuture<Acknowledge> jobSubmissionFuture = jarUploadFuture.thenCompose(jobGraph -> { + // we have to enable queued scheduling because slots will be allocated lazily --- End diff -- what do you mean? > Job submission via REST/dashboard does not work on Kubernetes > ------------------------------------------------------------- > > Key: FLINK-9211 > URL: https://issues.apache.org/jira/browse/FLINK-9211 > Project: Flink > Issue Type: Bug > Components: Client, Job-Submission, REST, Web Client > Affects Versions: 1.5.0 > Reporter: Aljoscha Krettek > Assignee: Chesnay Schepler > Priority: Blocker > Fix For: 1.5.0 > > > When setting up a cluster on Kubernets according to the documentation it is > possible to upload jar files but when trying to execute them you get an > exception like this: > {code} > org.apache.flink.runtime.rest.handler.RestHandlerException: > org.apache.flink.runtime.client.JobSubmissionException: Failed to submit > JobGraph. > at > org.apache.flink.runtime.webmonitor.handlers.JarRunHandler.lambda$handleRequest$2(JarRunHandler.java:113) > at > java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:870) > at > java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:852) > at > java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474) > at > java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977) > at > org.apache.flink.runtime.rest.RestClient.lambda$submitRequest$1(RestClient.java:196) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:680) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:603) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:563) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:424) > at > org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe$1.run(AbstractNioChannel.java:214) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.PromiseTask$RunnableAdapter.call(PromiseTask.java:38) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.ScheduledFutureTask.run(ScheduledFutureTask.java:120) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:357) > at > org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) > at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.util.concurrent.CompletionException: > org.apache.flink.runtime.client.JobSubmissionException: Failed to submit > JobGraph. > at > org.apache.flink.client.program.rest.RestClusterClient.lambda$submitJob$5(RestClusterClient.java:356) > ... 17 more > Caused by: org.apache.flink.runtime.client.JobSubmissionException: Failed to > submit JobGraph. > ... 18 more > Caused by: java.util.concurrent.CompletionException: > org.apache.flink.shaded.netty4.io.netty.channel.ConnectTimeoutException: > connection timed out: flink-jobmanager/10.105.154.28:8081 > at > java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292) > at > java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308) > at > java.util.concurrent.CompletableFuture.uniCompose(CompletableFuture.java:943) > at > java.util.concurrent.CompletableFuture$UniCompose.tryFire(CompletableFuture.java:926) > ... 15 more > Caused by: > org.apache.flink.shaded.netty4.io.netty.channel.ConnectTimeoutException: > connection timed out: flink-jobmanager/10.105.154.28:8081 > at > org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe$1.run(AbstractNioChannel.java:212) > ... 7 more > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)