[ https://issues.apache.org/jira/browse/FLINK-24496?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17427052#comment-17427052 ]
Sergey Nuyanzin commented on FLINK-24496: ----------------------------------------- Probably the issue is that here {{oldEntry.getExecutionGraphInfoFuture()}} is a {{java.util.concurrent.CompletableFuture}}. For this class {{cancel(true)}} and {{cancel(false)}} are the same It's also mentioned in javadoc for _java.util.concurrent.CompletableFuture#cancel_ {noformat} * @param mayInterruptIfRunning this value has no effect in this * implementation because interrupts are not used to control * processing. {noformat} https://github.com/openjdk/jdk11u/blob/73eef16128417f4a489c4dde47383bb4a00f39d4/src/java.base/share/classes/java/util/concurrent/CompletableFuture.java#L2389-L2391 > Querying job details in the REST API fails > -------------------------------------------- > > Key: FLINK-24496 > URL: https://issues.apache.org/jira/browse/FLINK-24496 > Project: Flink > Issue Type: Bug > Components: Runtime / Web Frontend > Affects Versions: 1.14.0 > Reporter: Juha > Priority: Major > > Hello, > In my setup there are three servers in a standalone the Flink cluster using > ZooKeeper HA Services running Flink 1.14.0. There's one TaskManager and one > JobManager on all servers. ZooKeeper is running on all servers. All the > servers have been just started and one simple job has been deployed. > One of the JobManagers is the leading JobManager. > If I query (curl) the leading job manager with /v1/jobs/<job_id>, the > response is correct. > But if I query a JobManager that is not the leading JobManager, the HTTP > request fails and the following can be seen in the logs. > {code} > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: [2021-10-08 09:28:09,998] > ERROR Unhandled exception. > (org.apache.flink.runtime.rest.handler.job.metrics.JobVertexWatermarksHandler:260) > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: > java.util.concurrent.CancellationException: null > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2396) > ~[?:?] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInternal(DefaultExecutionGraphCache.java:98) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInfo(DefaultExecutionGraphCache.java:67) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.handleRequest(AbstractExecutionGraphHandler.java:81) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.AbstractRestHandler.respondToRequest(AbstractRestHandler.java:83) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.AbstractHandler.respondAsLeader(AbstractHandler.java:195) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.lambda$channelRead0$0(LeaderRetrievalHandler.java:83) > ~[flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > java.util.Optional.ifPresent(Optional.java:183) [?:?] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.util.OptionalConsumer.ifPresent(OptionalConsumer.java:45) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:80) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:49) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.router.RouterHandler.routed(RouterHandler.java:115) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:94) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:55) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:238) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:71) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436) > [flink-dist_2> > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:296) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler.channelRead(CombinedChannelDuplexHandler.java:251) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:09 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:719) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:655) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:581) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > org.apache.flink.shaded.netty4.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) > [flink-dist_2.12-1.14.0.jar:1.14.0] > Oct 08 09:28:10 flink-perf-vy5z2cda-2 java[108]: at > java.lang.Thread.run(Thread.java:829) [?:?] > {code} > I'm not sure if this happens always, but I did manage to create two separate > Flink clusters and reproduce the issue. > As far as I know the job is running fine. > The issue doesn't exist on 1.13.0 or 1.13.2. > Regards > Juha -- This message was sent by Atlassian Jira (v8.3.4#803005)