Zdenek created FLINK-10160: ------------------------------ Summary: Index out of bound when serializing operator state Key: FLINK-10160 URL: https://issues.apache.org/jira/browse/FLINK-10160 Project: Flink Issue Type: Bug Components: Core Affects Versions: 1.4.2 Environment: Cloudera Hadoop 2.6.5
JVM: Java HotSpot(TM) 64-Bit Server VM - Oracle Corporation - 1.8/25.121-b13 Scala 2.11 Reporter: Zdenek Attachments: log.txt I am getting unexpected randomly happend error when checkpoint state is serialized to state backend (In-Memory). Source code usage: {code:java} env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) env.registerType(classOf[ImpressionMetric]) env.registerType(classOf[RecommendationMetric]) env.registerType(classOf[AdRequestMetric]) env.registerType(classOf[AdRequestStatusMetric]) val impressionsStream = env .addSource(impsSource) .map(imp => ImpressionMetric(imp)) .map(imp => imp.asInstanceOf[DomainMetric]) .name("Impressions") val recommendationsStream = env .addSource(recommendationsSource) .map(rcmd => RecommendationMetric(rcmd)) .map(rcmd => rcmd.asInstanceOf[DomainMetric]) .name("Recommendations") val adRequestsStream = env .addSource(adRequestsSource) .flatMap(new MapToAdRequestDomainMetrics()) .name("Ad requests") {code} Log with error, longer version in [^log.txt] {code:java} 2018-08-16 12:49:37,663 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 1 @ 1534416577551 2018-08-16 12:49:38,100 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 1 (1404248 bytes in 545 ms). 2018-08-16 12:57:54,217 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 2 @ 1534417074216 2018-08-16 12:57:54,368 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Completed checkpoint 2 (2199656 bytes in 152 ms). 2018-08-16 13:07:54,217 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 3 @ 1534417674216 2018-08-16 13:17:54,218 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint 3 expired before completing. 2018-08-16 13:17:54,220 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 4 @ 1534418274220 2018-08-16 13:27:54,220 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Checkpoint 4 expired before completing. 2018-08-16 13:27:54,222 INFO org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Triggering checkpoint 5 @ 1534418874221 2018-08-16 13:29:36,640 WARN org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Received late message for now expired checkpoint attempt 3 from e4ce857f2ef5dd4cc75a48b6fdb7b694 of job a98b29dcef6ad8eac28f8290034a0590. 2018-08-16 13:29:36,642 WARN org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Received late message for now expired checkpoint attempt 3 from 1db1e9c995bae21d6c81e828e11f50a6 of job a98b29dcef6ad8eac28f8290034a0590. 2018-08-16 13:29:36,661 WARN org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Received late message for now expired checkpoint attempt 4 from 0f62f47cdd04acf58cf8aa6c11288950 of job a98b29dcef6ad8eac28f8290034a0590. 2018-08-16 13:29:36,662 WARN org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Received late message for now expired checkpoint attempt 4 from e4ce857f2ef5dd4cc75a48b6fdb7b694 of job a98b29dcef6ad8eac28f8290034a0590. 2018-08-16 13:29:36,663 WARN org.apache.flink.runtime.checkpoint.CheckpointCoordinator - Received late message for now expired checkpoint attempt 3 from 0f62f47cdd04acf58cf8aa6c11288950 of job a98b29dcef6ad8eac28f8290034a0590. 2018-08-16 13:29:36,967 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph - Source: Custom Source -> Ad requests (1/2) (1db1e9c995bae21d6c81e828e11f50a6) switched from RUNNING to FAILED. java.lang.Exception: Error while triggering checkpoint 4 for Source: Custom Source -> Ad requests (1/2) at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1210) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.Exception: Could not perform checkpoint 4 for operator Source: Custom Source -> Ad requests (1/2). at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:544) at org.apache.flink.streaming.runtime.tasks.SourceStreamTask.triggerCheckpoint(SourceStreamTask.java:111) at org.apache.flink.runtime.taskmanager.Task$2.run(Task.java:1199) ... 5 more Caused by: java.lang.Exception: Could not complete snapshot 4 for operator Source: Custom Source -> Ad requests (1/2). at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:378) at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1089) at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1038) at org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:671) at org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:607) at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:538) ... 7 more Caused by: java.lang.ArrayIndexOutOfBoundsException: -1 at com.esotericsoftware.kryo.util.IntArray.add(IntArray.java:61) at com.esotericsoftware.kryo.Kryo.readReferenceOrNull(Kryo.java:800) at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:655) at org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.copy(KryoSerializer.java:189) at org.apache.flink.api.java.typeutils.runtime.TupleSerializer.copy(TupleSerializer.java:104) at org.apache.flink.api.java.typeutils.runtime.TupleSerializer.copy(TupleSerializer.java:30) at org.apache.flink.runtime.state.ArrayListSerializer.copy(ArrayListSerializer.java:74) at org.apache.flink.runtime.state.DefaultOperatorStateBackend$PartitionableListState.<init>(DefaultOperatorStateBackend.java:448) at org.apache.flink.runtime.state.DefaultOperatorStateBackend$PartitionableListState.deepCopy(DefaultOperatorStateBackend.java:460) at org.apache.flink.runtime.state.DefaultOperatorStateBackend.snapshot(DefaultOperatorStateBackend.java:220) at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:363) ... 12 more {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)