[
https://issues.apache.org/jira/browse/FLINK-24496?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17477696#comment-17477696
]
Peihui He commented on FLINK-24496:
-----------------------------------
This problem is caused by this FLINK-18783 change。
Through debugging, find the error:
{code:java}
2022-01-17 10:27:16,777 ERROR
org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler [] -
org.apache.flink.runtime.rpc.exceptions.RpcException: Could not deserialize the
serialized payload of RPC method : requestExecutionGraphInfo
java.util.concurrent.CompletionException:
org.apache.flink.runtime.rpc.exceptions.RpcException: Could not deserialize the
serialized payload of RPC method : requestExecutionGraphInfo
at
org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.deserializeValueIfNeeded(AkkaInvocationHandler.java:432)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.lambda$invokeRpc$0(AkkaInvocationHandler.java:256)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
~[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
~[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
~[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
~[?:1.8.0_261]
at
org.apache.flink.util.concurrent.FutureUtils.doForward(FutureUtils.java:1389)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$null$1(ClassLoadingUtils.java:93)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:68)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.lambda$guardCompletionWithContextClassLoader$2(ClassLoadingUtils.java:92)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
[?:1.8.0_261]
at
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
[?:1.8.0_261]
at
org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$1.onComplete(AkkaFutureUtils.java:54)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.OnComplete.internal(Future.scala:300)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.OnComplete.internal(Future.scala:297)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:224)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:221)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:60)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
org.apache.flink.runtime.concurrent.akka.AkkaFutureUtils$DirectExecutionContext.execute(AkkaFutureUtils.java:65)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:68)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1(Promise.scala:284)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
scala.concurrent.impl.Promise$DefaultPromise.$anonfun$tryComplete$1$adapted(Promise.scala:284)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:284)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:621)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.remote.DefaultMessageDispatcher.dispatch(Endpoint.scala:118)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at
akka.remote.EndpointReader$$anonfun$receive$2.applyOrElse(Endpoint.scala:1144)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.actor.Actor.aroundReceive(Actor.scala:537)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.actor.Actor.aroundReceive$(Actor.scala:535)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.remote.EndpointActor.aroundReceive(Endpoint.scala:540)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.actor.ActorCell.receiveMessage(ActorCell.scala:580)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.actor.ActorCell.invoke(ActorCell.scala:548)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.Mailbox.run(Mailbox.scala:231)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at akka.dispatch.Mailbox.exec(Mailbox.scala:243)
[flink-rpc-akka_b4f1dbdd-64c2-4f9c-906d-e73f283bd6fc.jar:1.14.0]
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
[?:1.8.0_261]
at
java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
[?:1.8.0_261]
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
[?:1.8.0_261]
at
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:172)
[?:1.8.0_261]
Caused by: org.apache.flink.runtime.rpc.exceptions.RpcException: Could not
deserialize the serialized payload of RPC method : requestExecutionGraphInfo
... 40 more
Caused by: java.lang.ClassNotFoundException:
org.apache.commons.math3.stat.descriptive.rank.Percentile
at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
~[?:1.8.0_261]
at java.lang.ClassLoader.loadClass(ClassLoader.java:418) ~[?:1.8.0_261]
at
org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentOnly(ComponentClassLoader.java:125)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:104)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at java.lang.ClassLoader.loadClass(ClassLoader.java:351) ~[?:1.8.0_261]
at java.lang.Class.forName0(Native Method) ~[?:1.8.0_261]
at java.lang.Class.forName(Class.java:348) ~[?:1.8.0_261]
at
org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:78)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1946)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1829)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2120)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2365)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2289)
~[?:1.8.0_261]
at
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2147)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1646)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:482)
~[?:1.8.0_261]
at java.io.ObjectInputStream.readObject(ObjectInputStream.java:440)
~[?:1.8.0_261]
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:617)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:602)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:589)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.rpc.akka.AkkaRpcSerializedValue.deserializeValue(AkkaRpcSerializedValue.java:55)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
at
org.apache.flink.runtime.rpc.akka.AkkaInvocationHandler.deserializeValueIfNeeded(AkkaInvocationHandler.java:427)
~[flink-dist_2.11-1.14.0.jar:1.14.0]
... 39 more {code}
This is because deserializeValueIfNeeded(resultValue, method) error.
To solve this problem, you can adjust the submoduleclassloader to add
"org.apache.commons.math3" to ownerFirstPackages.
> 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.20.1#820001)