[ https://issues.apache.org/jira/browse/FLINK-24163?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17410504#comment-17410504 ]
Yun Gao commented on FLINK-24163: --------------------------------- This one should be called by the test configuration. There are two problems: # The checkpoint timeout 5s is still not large enough, it might be caused when the unaligned checkpoint is enabled, the checkpoint performance id bounded by disk IO. # We also met with {code} java.util.concurrent.ExecutionException: java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=8641168, maxSize=5242880. Consider using a different checkpoint storage, like the FileSystemCheckpointStorage. at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) ~[?:1.8.0_292] at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1908) ~[?:1.8.0_292] at org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:66) ~[flink-streaming-java_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.finalizeNonFinishedSnapshots(AsyncCheckpointRunnable.java:177) ~[flink-streaming-java_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.run(AsyncCheckpointRunnable.java:124) [flink-streaming-java_2.11-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_292] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_292] at java.lang.Thread.run(Thread.java:748) [?:1.8.0_292] Caused by: java.io.IOException: Size of the state is larger than the maximum permitted memory-backed state. Size=8641168, maxSize=5242880. Consider using a different checkpoint storage, like the FileSystemCheckpointStorage. at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.checkSize(MemCheckpointStreamFactory.java:63) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetBytes(MemCheckpointStreamFactory.java:140) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory$MemoryCheckpointOutputStream.closeAndGetHandle(MemCheckpointStreamFactory.java:120) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.finishWriteAndResult(ChannelStateCheckpointWriter.java:218) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.doComplete(ChannelStateCheckpointWriter.java:240) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.lambda$complete$5(ChannelStateCheckpointWriter.java:202) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.runWithChecks(ChannelStateCheckpointWriter.java:296) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.complete(ChannelStateCheckpointWriter.java:200) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateCheckpointWriter.completeInput(ChannelStateCheckpointWriter.java:187) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.CheckpointInProgressRequest.execute(ChannelStateWriteRequest.java:212) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestDispatcherImpl.dispatchInternal(ChannelStateWriteRequestDispatcherImpl.java:85) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestDispatcherImpl.dispatch(ChannelStateWriteRequestDispatcherImpl.java:62) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestExecutorImpl.loop(ChannelStateWriteRequestExecutorImpl.java:96) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] at org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestExecutorImpl.run(ChannelStateWriteRequestExecutorImpl.java:75) ~[flink-runtime-1.14-SNAPSHOT.jar:1.14-SNAPSHOT] ... 1 more {code} This might be caused by we have channel state with unaligned checkpoints. I'll open a PR to increase the checkpoint timeout and change to another statebackend type. > PartiallyFinishedSourcesITCase fails due to timeout > --------------------------------------------------- > > Key: FLINK-24163 > URL: https://issues.apache.org/jira/browse/FLINK-24163 > Project: Flink > Issue Type: Bug > Components: API / DataStream > Affects Versions: 1.14.0, 1.15.0 > Reporter: Xintong Song > Priority: Blocker > Labels: test-stability > Fix For: 1.14.0, 1.15.0 > > > https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=23529&view=logs&j=4d4a0d10-fca2-5507-8eed-c07f0bdf4887&t=7b25afdf-cc6c-566f-5459-359dc2585798&l=10996 > {code} > Sep 04 04:35:28 [ERROR] Tests run: 6, Failures: 0, Errors: 1, Skipped: 0, > Time elapsed: 155.236 s <<< FAILURE! - in > org.apache.flink.runtime.operators.lifecycle.PartiallyFinishedSourcesITCase > Sep 04 04:35:28 [ERROR] test[complex graph ALL_SUBTASKS, failover: false] > Time elapsed: 65.999 s <<< ERROR! > Sep 04 04:35:28 java.util.concurrent.TimeoutException: Condition was not met > in given timeout. > Sep 04 04:35:28 at > org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:164) > Sep 04 04:35:28 at > org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:142) > Sep 04 04:35:28 at > org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition(CommonTestUtils.java:134) > Sep 04 04:35:28 at > org.apache.flink.runtime.testutils.CommonTestUtils.waitForSubtasksToFinish(CommonTestUtils.java:297) > Sep 04 04:35:28 at > org.apache.flink.runtime.operators.lifecycle.TestJobExecutor.waitForSubtasksToFinish(TestJobExecutor.java:219) > Sep 04 04:35:28 at > org.apache.flink.runtime.operators.lifecycle.PartiallyFinishedSourcesITCase.test(PartiallyFinishedSourcesITCase.java:82) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)