[ https://issues.apache.org/jira/browse/FLINK-10419?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16678328#comment-16678328 ]
ASF GitHub Bot commented on FLINK-10419: ---------------------------------------- NicoK closed pull request #7033: [FLINK-10419][checkpoint] fix ClassNotFoundException while deserializing user exceptions URL: https://github.com/apache/flink/pull/7033 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java index 7b0b55c9a1b..f8f3c0117bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/DeclineCheckpoint.java @@ -19,12 +19,6 @@ package org.apache.flink.runtime.messages.checkpoint; import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.checkpoint.decline.AlignmentLimitExceededException; -import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException; -import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineSubsumedException; -import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotCheckpointingException; -import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotReadyException; -import org.apache.flink.runtime.checkpoint.decline.InputEndOfStreamException; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.util.SerializedThrowable; @@ -38,7 +32,7 @@ private static final long serialVersionUID = 2094094662279578953L; - /** The reason why the checkpoint was declined */ + /** The reason why the checkpoint was declined. */ private final Throwable reason; public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId, long checkpointId) { @@ -47,19 +41,12 @@ public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId, long che public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId, long checkpointId, Throwable reason) { super(job, taskExecutionId, checkpointId); - - if (reason == null || - reason.getClass() == AlignmentLimitExceededException.class || - reason.getClass() == CheckpointDeclineOnCancellationBarrierException.class || - reason.getClass() == CheckpointDeclineSubsumedException.class || - reason.getClass() == CheckpointDeclineTaskNotCheckpointingException.class || - reason.getClass() == CheckpointDeclineTaskNotReadyException.class || - reason.getClass() == InputEndOfStreamException.class) - { - // null or known common exceptions that cannot reference any dynamically loaded code + + if (reason == null) { this.reason = reason; } else { - // some other exception. replace with a serialized throwable, to be on the safe side + // exceptions may reference dynamically loaded code (exception itself, cause, suppressed) + // -> replace with a serialized throwable, to be on the safe side this.reason = new SerializedThrowable(reason); } } @@ -68,7 +55,7 @@ public DeclineCheckpoint(JobID job, ExecutionAttemptID taskExecutionId, long che /** * Gets the reason why the checkpoint was declined. - * + * * @return The reason why the checkpoint was declined */ public Throwable getReason() { ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > ClassNotFoundException while deserializing user exceptions from checkpointing > ----------------------------------------------------------------------------- > > Key: FLINK-10419 > URL: https://issues.apache.org/jira/browse/FLINK-10419 > Project: Flink > Issue Type: Bug > Components: Distributed Coordination, State Backends, Checkpointing > Affects Versions: 1.5.0, 1.5.1, 1.5.2, 1.5.3, 1.5.4, 1.6.0, 1.6.1, 1.7.0 > Reporter: Nico Kruber > Assignee: Nico Kruber > Priority: Major > Labels: pull-request-available > Fix For: 1.7.0 > > > It seems that somewhere in the operator's failure handling, we hand a > user-code exception to the checkpoint coordinator via Java serialization but > it will then fail during the de-serialization because the class is not > available. This will result in the following error shadowing the real one: > {code} > java.lang.ClassNotFoundException: > org.apache.flink.streaming.connectors.kafka.FlinkKafka011Exception > at java.net.URLClassLoader.findClass(URLClassLoader.java:381) > at java.lang.ClassLoader.loadClass(ClassLoader.java:424) > at sun.misc.Launcher.loadClass(Launcher.java:338) > at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.flink.util.InstantiationUtil.resolveClass(InstantiationUtil.java:76) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1859) > at > java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1745) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2033) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1567) > at > java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2278) > at > java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:557) > at java.lang.Throwable.readObject(Throwable.java:914) > at sun.reflect.GeneratedMethodAccessor158.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1158) > at > java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2169) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2060) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1567) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:427) > at > org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation.readObject(RemoteRpcInvocation.java:222) > at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at > java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1158) > at > java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2169) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2060) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1567) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:427) > at > org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:502) > at > org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:489) > at > org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:477) > at > org.apache.flink.util.SerializedValue.deserializeValue(SerializedValue.java:58) > at > org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation.deserializeMethodInvocation(RemoteRpcInvocation.java:118) > at > org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation.getMethodName(RemoteRpcInvocation.java:59) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:214) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:162) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:70) > at > org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142) > at > org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40) > at > akka.actor.UntypedActor3728anonfun.applyOrElse(UntypedActor.scala:165) > at akka.actor.Actor.aroundReceive(Actor.scala:502) > at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95) > at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526) > at akka.actor.ActorCell.invoke(ActorCell.scala:495) > at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257) > at akka.dispatch.Mailbox.run(Mailbox.scala:224) > at akka.dispatch.Mailbox.exec(Mailbox.scala:234) > at > scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) > at > scala.concurrent.forkjoin.ForkJoinPool.runTask(ForkJoinPool.java:1339) > at > scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) > at > scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)