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

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

GitHub user rmetzger opened a pull request:

    https://github.com/apache/flink/pull/1048

    [FLINK-2543] Fix user object deserialization for user state

    File-based state handles were using the system classloader to deserialize 
the state object.
    
    Exceptions send from the JobManager to the JobClient were relying on Akka's 
JavaSerialization, which does not have access to the user code classloader.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/rmetzger/flink flink2543

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1048.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1048
    
----
commit f5a642267a5ce69018d900398e4eb5134b2ea747
Author: Robert Metzger <rmetz...@apache.org>
Date:   2015-08-18T16:15:40Z

    [FLINK-2543] Fix user object deserialization for file-based state handles. 
Send exceptions from JM --> JC in serialized form
    
    Exceptions send from the JobManager to the JobClient were relying on Akka's 
JavaSerialization, which does not have access to the user code classloader.

----


> 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
>
> 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