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

Chesnay Schepler commented on FLINK-24550:
------------------------------------------

The deserialization of the ArchivedExecutionGraph fails on standby job managers 
due to a classloading issue:

{code}
Caused by: org.apache.flink.runtime.rpc.exceptions.RpcException: Could not 
deserialize the serialized payload of RPC method : requestExecutionGraphInfo
        ... 41 more
Caused by: java.lang.ClassNotFoundException: 
org.apache.commons.math3.stat.descriptive.rank.Percentile
{code}

> Can not access job information from a standby jobmanager UI
> -----------------------------------------------------------
>
>                 Key: FLINK-24550
>                 URL: https://issues.apache.org/jira/browse/FLINK-24550
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination, Runtime / Web Frontend
>    Affects Versions: 1.14.0
>            Reporter: Dawid Wysakowicz
>            Assignee: Chesnay Schepler
>            Priority: Blocker
>             Fix For: 1.15.0, 1.14.1
>
>
> One can not access the "running jobs" section (if a job is running) or if the 
> job is completed it can not access the job page. Moreover the overview 
> section does not work in the standby manager if a job is running. The active 
> jobmanager UI works just fine.
> {code}
> 2021-10-14 15:45:11,483 ERROR 
> org.apache.flink.runtime.rest.handler.job.JobExceptionsHandler [] - Unhandled 
> exception.
> java.util.concurrent.CancellationException: null
>         at 
> java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2263) 
> ~[?:1.8.0_231]
>         at 
> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInternal(DefaultExecutionGraphCache.java:98)
>  ~[flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache.getExecutionGraphInfo(DefaultExecutionGraphCache.java:67)
>  ~[flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.handleRequest(AbstractExecutionGraphHandler.java:81)
>  ~[flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.AbstractRestHandler.respondToRequest(AbstractRestHandler.java:83)
>  ~[flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.AbstractHandler.respondAsLeader(AbstractHandler.java:195)
>  ~[flink-dist_2.12-1.14.0.jar:1.14.0]
>         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]
>         at java.util.Optional.ifPresent(Optional.java:159) [?:1.8.0_231]
>         at 
> org.apache.flink.util.OptionalConsumer.ifPresent(OptionalConsumer.java:45) 
> [flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:80)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.LeaderRetrievalHandler.channelRead0(LeaderRetrievalHandler.java:49)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         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]
>         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]
>         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]
>         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]
>         at 
> org.apache.flink.runtime.rest.handler.router.RouterHandler.routed(RouterHandler.java:115)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:94)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.handler.router.RouterHandler.channelRead0(RouterHandler.java:55)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         at 
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:238)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         at 
> org.apache.flink.runtime.rest.FileUploadHandler.channelRead0(FileUploadHandler.java:71)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         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]
>         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]
>         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]
>         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]
>         at 
> org.apache.flink.shaded.netty4.io.netty.channel.CombinedChannelDuplexHandler$DelegatingChannelHandlerContext.fireChannelRead(CombinedChannelDuplexHandler.java:436)
>  [flink-dist_2.12-1.14.0.jar:1.14.0]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         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]
>         at java.lang.Thread.run(Thread.java:748) [?:1.8.0_231]
> {code}
> It seems to be working just fine in 1.13.2
> Reported in the ML:  
> https://lists.apache.org/thread.html/r69646f1c943846ed07f9ff80232c8d0cea31222191354871f914484c%40%3Cuser.flink.apache.org%3E



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to