[
https://issues.apache.org/jira/browse/FLINK-7760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Shashank Agarwal reopened FLINK-7760:
-------------------------------------
Restore failing from savepoint too in 1.4.0 Printing following logs :
{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}
> Restore failing from external checkpointing metadata.
> -----------------------------------------------------
>
> Key: FLINK-7760
> URL: https://issues.apache.org/jira/browse/FLINK-7760
> Project: Flink
> Issue Type: Sub-task
> Components: CEP, State Backends, Checkpointing
> Affects Versions: 1.3.2
> Environment: Yarn, Flink 1.3.2, HDFS, FsStateBackend
> Reporter: Shashank Agarwal
> Priority: Blocker
> Fix For: 1.4.0, 1.5.0, 1.4.1
>
>
> My job failed due to failure of cassandra. I have enabled
> ExternalizedCheckpoints. But when job tried to restore from that checkpoint
> it's failing continuously with following error.
> {code:java}
> 2017-10-04 09:39:20,611 INFO org.apache.flink.runtime.taskmanager.Task
> - KeyedCEPPatternOperator -> Map (1/2)
> (8ff7913f820ead571c8b54ccc6b16045) switched from RUNNING to FAILED.
> java.lang.IllegalStateException: Could not initialize keyed state backend.
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:321)
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:217)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:676)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:663)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:252)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.StreamCorruptedException: invalid type code: 00
> at
> java.io.ObjectInputStream$BlockDataInputStream.readBlockHeader(ObjectInputStream.java:2519)
> at
> java.io.ObjectInputStream$BlockDataInputStream.refill(ObjectInputStream.java:2553)
> at
> java.io.ObjectInputStream$BlockDataInputStream.skipBlockData(ObjectInputStream.java:2455)
> at java.io.ObjectInputStream.skipCustomData(ObjectInputStream.java:1951)
> at
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1621)
> at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518)
> at
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1623)
> 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.cep.nfa.NFA$NFASerializer.deserializeCondition(NFA.java:1211)
> at
> org.apache.flink.cep.nfa.NFA$NFASerializer.deserializeStates(NFA.java:1169)
> at org.apache.flink.cep.nfa.NFA$NFASerializer.deserialize(NFA.java:957)
> at org.apache.flink.cep.nfa.NFA$NFASerializer.deserialize(NFA.java:852)
> at
> org.apache.flink.runtime.state.heap.StateTableByKeyGroupReaders$StateTableByKeyGroupReaderV2V3.readMappingsInKeyGroup(StateTableByKeyGroupReaders.java:132)
> at
> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend.restorePartitionedState(HeapKeyedStateBackend.java:518)
> at
> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend.restore(HeapKeyedStateBackend.java:397)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.createKeyedStateBackend(StreamTask.java:772)
> at
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.initKeyedState(AbstractStreamOperator.java:311)
> ... 6 more
> {code}
> I have tried to start new job also after failure with parameter {code:java}
> -s [checkpoint meta data path]{code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)