[ 
https://issues.apache.org/jira/browse/FLINK-4140?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Stefan Richter updated FLINK-4140:
----------------------------------
    Assignee: Ufuk Celebi  (was: Stefan Richter)

> CheckpointCoordinator fails to discard completed checkpoint
> -----------------------------------------------------------
>
>                 Key: FLINK-4140
>                 URL: https://issues.apache.org/jira/browse/FLINK-4140
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.0.3
>            Reporter: Stefan Richter
>            Assignee: Ufuk Celebi
>
> Running a job in HA mode I saw the following warning in the job manager logs. 
> The warning appeared after the job was restarted due to a master failure. 
> I've skimmed the code and it looks like the user code class loader is used 
> everywhere when discarding the checkpoint, but something seems to not work as 
> expected (otherwise the warning should not appear).
> {code}
> 2016-07-01 13:08:33,218 WARN  
> org.apache.flink.runtime.checkpoint.SubtaskState              - Failed to 
> discard checkpoint state: StateForTask(Size: 4, Duration: 2012, State: 
> SerializedValue)
> java.lang.ClassNotFoundException: da.testing.State
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>         at java.lang.Class.forName0(Native Method)
>         at java.lang.Class.forName(Class.java:278)
>         at 
> org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:64)
>         at 
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1620)
>         at 
> java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1521)
>         at java.io.ObjectInputStream.readClass(ObjectInputStream.java:1486)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1336)
>         at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2016)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1940)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2016)
>         at 
> java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:503)
>         at 
> org.apache.flink.api.common.state.StateDescriptor.readObject(StateDescriptor.java:268)
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at 
> java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1058)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1907)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2016)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1940)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at java.io.ObjectInputStream.readObject(ObjectInputStream.java:373)
>         at java.util.HashMap.readObject(HashMap.java:1180)
>         at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at 
> java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1058)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1907)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2016)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1940)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at java.io.ObjectInputStream.readArray(ObjectInputStream.java:1714)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1347)
>         at 
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2016)
>         at 
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1940)
>         at 
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1806)
>         at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
>         at java.io.ObjectInputStream.readObject(ObjectInputStream.java:373)
>         at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:290)
>         at 
> org.apache.flink.util.SerializedValue.deserializeValue(SerializedValue.java:58)
>         at 
> org.apache.flink.runtime.checkpoint.SubtaskState.discard(SubtaskState.java:86)
>         at 
> org.apache.flink.runtime.checkpoint.TaskState.discard(TaskState.java:147)
>         at 
> org.apache.flink.runtime.checkpoint.CompletedCheckpoint.discard(CompletedCheckpoint.java:102)
>         at 
> org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore$1.processResult(ZooKeeperCompletedCheckpointStore.java:269)
>         at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.sendToBackgroundCallback(CuratorFrameworkImpl.java:728)
>         at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.CuratorFrameworkImpl.processBackgroundOperation(CuratorFrameworkImpl.java:505)
>         at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.NamespaceFacade.processBackgroundOperation(NamespaceFacade.java:118)
>         at 
> org.apache.flink.shaded.org.apache.curator.framework.imps.DeleteBuilderImpl$2.processResult(DeleteBuilderImpl.java:163)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:609)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to