Shashank Agarwal created FLINK-8319:
---------------------------------------

             Summary: Savepoint restore failing in CEP
                 Key: FLINK-8319
                 URL: https://issues.apache.org/jira/browse/FLINK-8319
             Project: Flink
          Issue Type: Bug
          Components: CEP, State Backends, Checkpointing, YARN
    Affects Versions: 1.4.0
         Environment: Yarn Cluster
            Reporter: Shashank Agarwal
             Fix For: 1.5.0, 1.4.1


I have reported some bugs before also in 1.3.2 but this time error is different 
while restoring savepoint or checkpoint.

https://issues.apache.org/jira/browse/FLINK-7760


{code}
java.lang.IllegalStateException: Could not initialize keyed state backend.
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:293)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:225)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:692)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:679)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.InvalidClassException: 
org.apache.flink.cep.scala.pattern.Pattern$$anon$3; invalid descriptor for 
field 
        at java.io.ObjectStreamClass.readNonProxy(ObjectStreamClass.java:723)
        at 
java.io.ObjectInputStream.readClassDescriptor(ObjectInputStream.java:833)
        at 
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1609)
        at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1521)
        at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1781)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1353)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:373)
        at 
org.apache.flink.cep.nfa.NFA$NFASerializer.deserializeCondition(NFA.java:1171)
        at 
org.apache.flink.cep.nfa.NFA$NFASerializer.deserializeStates(NFA.java:1129)
        at org.apache.flink.cep.nfa.NFA$NFASerializer.deserialize(NFA.java:917)
        at org.apache.flink.cep.nfa.NFA$NFASerializer.deserialize(NFA.java:820)
        at 
org.apache.flink.runtime.state.heap.StateTableByKeyGroupReaders$StateTableByKeyGroupReaderV2V3.readMappingsInKeyGroup(StateTableByKeyGroupReaders.java:133)
        at 
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend.restorePartitionedState(HeapKeyedStateBackend.java:575)
        at 
org.apache.flink.runtime.state.heap.HeapKeyedStateBackend.restore(HeapKeyedStateBackend.java:446)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.createKeyedStateBackend(StreamTask.java:773)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:283)
        ... 6 more
Caused by: java.lang.IllegalArgumentException: illegal signature
        at java.io.ObjectStreamField.<init>(ObjectStreamField.java:122)
        at java.io.ObjectStreamClass.readNonProxy(ObjectStreamClass.java:721)
        ... 21 more
{code}




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to