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

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

Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1048#issuecomment-135793797
  
    Hmm... the more I work on it, the trickier it appears.
    
    It is efficient when done rigth, but one needs to watch carefully at what 
places to wrap an exception and when not.
    
    The solution to always serialize/deserialize manually during transport is 
far easier, actually...


> State handling does not support deserializing classes through the 
> UserCodeClassloader
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-2543
>                 URL: https://issues.apache.org/jira/browse/FLINK-2543
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 0.9, 0.10
>            Reporter: Robert Metzger
>            Assignee: Robert Metzger
>            Priority: Blocker
>             Fix For: 0.10
>
>
> The current implementation of the state checkpointing does not support custom 
> classes, because the UserCodeClassLoader is not used to deserialize the state.
> {code}
> Error: java.lang.RuntimeException: Failed to deserialize state handle and 
> setup initial operator state.
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:544)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.ClassNotFoundException: 
> com.ottogroup.bi.searchlab.searchsessionizer.OperatorState
> at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
> at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
> at java.lang.Class.forName0(Native Method)
> at java.lang.Class.forName(Class.java:348)
> at java.io.ObjectInputStream.resolveClass(ObjectInputStream.java:626)
> at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613)
> at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518)
> at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774)
> at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
> at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)
> at 
> org.apache.flink.runtime.state.ByteStreamStateHandle.getState(ByteStreamStateHandle.java:63)
> at 
> org.apache.flink.runtime.state.ByteStreamStateHandle.getState(ByteStreamStateHandle.java:33)
> at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreInitialState(AbstractUdfStreamOperator.java:83)
> at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.setInitialState(StreamTask.java:276)
> at 
> org.apache.flink.runtime.state.StateUtils.setOperatorState(StateUtils.java:51)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:541)
> {code}
> The issue has been reported by a user: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Custom-Class-for-state-checkpointing-td2415.html



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

Reply via email to