You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Aljoscha Krettek (JIRA)" <ji...@apache.org> on 2017/04/18 12:36:41 UTC

[jira] [Created] (FLINK-6318) NFA serialisation doesn't work with TypeSerializers that load classes

Aljoscha Krettek created FLINK-6318:
---------------------------------------

             Summary: NFA serialisation doesn't work with TypeSerializers that load classes
                 Key: FLINK-6318
                 URL: https://issues.apache.org/jira/browse/FLINK-6318
             Project: Flink
          Issue Type: Improvement
          Components: CEP
            Reporter: Aljoscha Krettek


This is from a user reported stack trace:

{code}
04/12/2017 10:05:04	Job execution switched to status FAILING.
java.lang.RuntimeException: Could not deserialize NFA.
	at org.apache.flink.cep.nfa.NFA$Serializer.deserialize(NFA.java:538)
	at org.apache.flink.cep.nfa.NFA$Serializer.deserialize(NFA.java:469)
	at org.apache.flink.contrib.streaming.state.RocksDBValueState.value(RocksDBValueState.java:81)
	at org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator.getNFA(AbstractKeyedCEPPatternOperator.java:124)
	at org.apache.flink.cep.operator.AbstractCEPBasePatternOperator.processElement(AbstractCEPBasePatternOperator.java:72)
	at org.apache.flink.cep.operator.AbstractKeyedCEPPatternOperator.processElement(AbstractKeyedCEPPatternOperator.java:162)
	at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:185)
	at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:272)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655)
	at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.ClassNotFoundException: co.ronak.nto.Job$$anon$18$$anon$21$$anon$3
	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.defaultReadFields(ObjectInputStream.java:2000)
	at java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:501)
	at org.apache.flink.api.scala.typeutils.TraversableSerializer.readObject(TraversableSerializer.scala:53)
	at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:497)
	at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1900)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1924)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
	at java.io.ObjectInputStream.readArray(ObjectInputStream.java:1707)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1345)
	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1924)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
	at java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:501)
	at org.apache.flink.cep.NonDuplicatingTypeSerializer.readObject(NonDuplicatingTypeSerializer.java:190)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:497)
	at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1900)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
	at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2000)
	at java.io.ObjectInputStream.defaultReadObject(ObjectInputStream.java:501)
	at org.apache.flink.cep.nfa.NFA.readObject(NFA.java:394)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:497)
	at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017)
	at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1900)
	at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1801)
	at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351)
	at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371)
	at org.apache.flink.cep.nfa.NFA$Serializer.deserialize(NFA.java:535)
	... 10 more
{code}

The problem seems to be that {{NFA.readObject()}} internally uses a {{TypeSerializer}} to read some other stuff and wrapped in those {{TypeSerializers}} might be code that tries to resolve classes and there we then don't use the user-code class loader.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)