You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "Pawel Bartoszek (JIRA)" <ji...@apache.org> on 2017/08/08 14:54:00 UTC

[jira] [Created] (BEAM-2752) Job fails to checkpoint with kinesis stream as an input for Flink job

Pawel Bartoszek created BEAM-2752:
-------------------------------------

             Summary: Job fails to checkpoint with kinesis stream as an input for Flink job
                 Key: BEAM-2752
                 URL: https://issues.apache.org/jira/browse/BEAM-2752
             Project: Beam
          Issue Type: Bug
          Components: sdk-java-extensions
    Affects Versions: 2.0.0
            Reporter: Pawel Bartoszek
            Assignee: Davor Bonaci
            Priority: Minor


Our job is reading from kinesis stream as a job input. Quiet often when the job is checkpointing for the first time the exception is thrown:


{code:java}
Our job is reading from kinesis stream as a job input. Quiet often when the job is checkpointing for the first time the exception is thrown.

The scenario the produces the exception:

# Upload a new jar file with job logic
# Start new job
# Stop the job with savepoint that is written to s3
# Upload a new jar file with job logic(in this case the jar contains the same code - but our pipeline generates new jar file name for every build)
# Start a new job from savepoint
# The first checkpoint fails causing the job to be cancelled

If the job is started without passing savepoint the checkpointing works fine.

Other information:

Flink version 1.2.1
Beam 2.0.0
Flink Parallelism - 20 slots 
Number of task managers - 4



{code:java}
java.lang.Exception: Error while triggering checkpoint 59 for Source: Read(KinesisSource) -> Flat Map -> ParMultiDo(KinesisExtractor) -> Flat Map -> ParMultiDo(StringToRecord) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToRRecord) -> Flat Map -> ParMultiDo(AddTimestamps) -> Flat Map -> xxxx.yyyy.GroupByOneMinuteWindow GROUP RDOTRECORDS BY ONE MINUTE WINDOWS/Window.Assign.out -> (ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToSomeKey) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(ToCompositeKey) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ApplyShardingKey) -> Flat Map -> ToKeyedWorkItem) (1/20)
	at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1136)
	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:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.Exception: Could not perform checkpoint 59 for operator Source: Read(KinesisSource) -> Flat Map -> ParMultiDo(KinesisExtractor) -> Flat Map -> ParMultiDo(StringToRecord) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToRRecord) -> Flat Map -> ParMultiDo(AddTimestamps) -> Flat Map -> xxxx.yyyy.GroupByOneMinuteWindow GROUP RDOTRECORDS BY ONE MINUTE WINDOWS/Window.Assign.out -> (ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToSomeKey) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(ToCompositeKey) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ApplyShardingKey) -> Flat Map -> ToKeyedWorkItem) (1/20).
	at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:524)
	at org.apache.flink.runtime.taskmanager.Task$3.run(Task.java:1125)
	... 5 more
Caused by: java.lang.Exception: Could not complete snapshot 59 for operator Source: Read(KinesisSource) -> Flat Map -> ParMultiDo(KinesisExtractor) -> Flat Map -> ParMultiDo(StringToRecord) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToRRecord) -> Flat Map -> ParMultiDo(AddTimestamps) -> Flat Map -> xxxx.yyyy.GroupByOneMinuteWindow GROUP RDOTRECORDS BY ONE MINUTE WINDOWS/Window.Assign.out -> (ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ToSomeKey) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(ToCompositeKey) -> Flat Map -> ParMultiDo(Anonymous) -> Flat Map -> ToKeyedWorkItem, ParMultiDo(Anonymous) -> Flat Map -> ParMultiDo(ApplyShardingKey) -> Flat Map -> ToKeyedWorkItem) (1/20).
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:379)
	at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1157)
	at org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1090)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:630)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:575)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:518)
	... 6 more
Caused by: java.util.ConcurrentModificationException
	at java.util.ArrayDeque$DeqIterator.next(ArrayDeque.java:643)
	at org.apache.beam.sdks.java.io.kinesis.repackaged.com.google.common.collect.TransformedIterator.next(TransformedIterator.java:47)
	at org.apache.beam.sdks.java.io.kinesis.repackaged.com.google.common.collect.ImmutableCollection$Builder.addAll(ImmutableCollection.java:409)
	at org.apache.beam.sdks.java.io.kinesis.repackaged.com.google.common.collect.ImmutableList$Builder.addAll(ImmutableList.java:699)
	at org.apache.beam.sdks.java.io.kinesis.repackaged.com.google.common.collect.ImmutableList.copyOf(ImmutableList.java:256)
	at org.apache.beam.sdks.java.io.kinesis.repackaged.com.google.common.collect.ImmutableList.copyOf(ImmutableList.java:209)
	at org.apache.beam.sdk.io.kinesis.KinesisReaderCheckpoint.<init>(KinesisReaderCheckpoint.java:44)
	at org.apache.beam.sdk.io.kinesis.KinesisReaderCheckpoint.asCurrentStateOf(KinesisReaderCheckpoint.java:49)
	at org.apache.beam.sdk.io.kinesis.KinesisReader.getCheckpointMark(KinesisReader.java:137)
	at org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.snapshotState(UnboundedSourceWrapper.java:379)
	at org.apache.flink.streaming.api.functions.util.StreamingFunctionUtils.trySnapshotFunctionState(StreamingFunctionUtils.java:118)
	at org.apache.flink.streaming.api.functions.util.StreamingFunctionUtils.snapshotFunctionState(StreamingFunctionUtils.java:99)
	at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.snapshotState(AbstractUdfStreamOperator.java:100)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:357)
	... 11 more

{code}




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