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

ASF GitHub Bot commented on FLINK-10419:
----------------------------------------

tillrohrmann commented on a change in pull request #7082: [FLINK-10419] Using 
DeclineCheckpoint message class when invoking RPC declineCheckpoint
URL: https://github.com/apache/flink/pull/7082#discussion_r232988684
 
 

 ##########
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
 ##########
 @@ -248,6 +257,88 @@ public static void teardownClass() {
                }
        }
 
+       @Test
+       public void testDeclineCheckpointInvocationWithUserException() throws 
Exception {
+               RpcService rpcService1 = null;
+               RpcService rpcService2 = null;
+               try {
+                       final ActorSystem actorSystem1 = 
AkkaUtils.createDefaultActorSystem();
+                       final ActorSystem actorSystem2 = 
AkkaUtils.createDefaultActorSystem();
+
+                       rpcService1 = new AkkaRpcService(actorSystem1, 
testingTimeout);
+                       rpcService2 = new AkkaRpcService(actorSystem2, 
testingTimeout);
+
+                       final CompletableFuture<Throwable> 
declineCheckpointMessageFuture = new CompletableFuture<>();
+
+                       final JobManagerSharedServices jobManagerSharedServices 
= new TestingJobManagerSharedServicesBuilder().build();
+                       final JobMasterConfiguration jobMasterConfiguration = 
JobMasterConfiguration.fromConfiguration(configuration);
+                       final JobMaster jobMaster = new JobMaster(
+                               rpcService1,
+                               jobMasterConfiguration,
+                               jmResourceId,
+                               jobGraph,
+                               haServices,
+                               
DefaultSlotPoolFactory.fromConfiguration(configuration, rpcService1),
+                               jobManagerSharedServices,
+                               heartbeatServices,
+                               blobServer,
+                               
UnregisteredJobManagerJobMetricGroupFactory.INSTANCE,
+                               new NoOpOnCompletionActions(),
+                               testingFatalErrorHandler,
+                               JobMasterTest.class.getClassLoader()) {
+                               @Override
+                               public void declineCheckpoint(DeclineCheckpoint 
declineCheckpoint) {
+                                       
declineCheckpointMessageFuture.complete(declineCheckpoint.getReason());
+                               }
+                       };
+
+                       jobMaster.start(jobMasterId, testingTimeout).get();
+
+                       final String className = "UserException";
+                       final URLClassLoader userClassLoader = 
ClassLoaderUtils.compileAndLoadJava(
+                               temporaryFolder.newFolder(),
+                               className + ".java",
+                               String.format("public class %s extends 
RuntimeException { public %s() {super(\"UserMessage\");} }",
+                                       className,
+                                       className));
+
+                       Throwable userException = (Throwable) 
Class.forName(className, false, userClassLoader).newInstance();
+
+                       CompletableFuture<JobMasterGateway> jobMasterGateway =
+                               rpcService2.connect(jobMaster.getAddress(), 
jobMaster.getFencingToken(), JobMasterGateway.class);
+
+                       jobMasterGateway.thenAccept(gateway -> {
+                               try {
+                                       gateway.declineCheckpoint(new 
DeclineCheckpoint(
 
 Review comment:
   We could create the `DeclineCheckpoint` instance outside of the future 
callback. Then we would not need to handle the exception.

----------------------------------------------------------------
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: Dawid Wysakowicz
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.5.6, 1.6.3, 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)

Reply via email to