[
https://issues.apache.org/jira/browse/FLINK-30785?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17682143#comment-17682143
]
Roman Khachatryan commented on FLINK-30785:
-------------------------------------------
Thanks [~mapohl] ,
The test failed because the logs contained exceptions. Those are
CheckpointException and CancellationException.
According to [this
pattern|https://github.com/apache/flink/blob/c9e87fe410c42f7e7c19c81456d4212a58564f5e/flink-end-to-end-tests/test-scripts/common.sh#L405],
they both should be ignored.
However, one of the stacktraces contained "completeExceptionally", which I
think is not ignored:
{code:java}
2023-01-25 02:47:38,530 INFO
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable [] - Source:
EventSource -> Timestamps/Watermarks (1/2)#0 - asynchronous part of checkpoint
1 could
not be completed.
java.util.concurrent.CancellationException: null
at
java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2276)
~[?:1.8.0_352]
at
org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:78)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.api.operators.OperatorSnapshotFutures.lambda$cancel$0(OperatorSnapshotFutures.java:173)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.shaded.guava30.com.google.common.io.Closer.close(Closer.java:213)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.api.operators.OperatorSnapshotFutures.cancel(OperatorSnapshotFutures.java:185)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.cleanup(AsyncCheckpointRunnable.java:391)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.AsyncCheckpointRunnable.close(AsyncCheckpointRunnable.java:356)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:295)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at org.apache.flink.util.IOUtils.closeAllQuietly(IOUtils.java:282)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.cancel(SubtaskCheckpointCoordinatorImpl.java:547)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$cancel$10(StreamTask.java:985)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
at
java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
~[?:1.8.0_352]
at
java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
~[?:1.8.0_352]
at
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
~[?:1.8.0_352]
at
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
~[?:1.8.0_352]
at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:344)
~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]{code}
So this looks like a false positive. I think we should add
"completeExceptionally" to the ignore pattern.
WDYT?
cc: [~chesnay]
> RocksDB Memory Management end-to-end test failed due to unexpected exception
> ----------------------------------------------------------------------------
>
> Key: FLINK-30785
> URL: https://issues.apache.org/jira/browse/FLINK-30785
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.17.0
> Reporter: Matthias Pohl
> Priority: Critical
> Labels: test-stability
>
> We see a test instability with {{RocksDB Memory Management end-to-end test}}.
> The test failed because an exception was detected in the logs:
> {code}
> 2023-01-25T02:47:38.7172354Z Jan 25 02:47:38 Checking for errors...
> 2023-01-25T02:47:39.1661969Z Jan 25 02:47:39 No errors in log files.
> 2023-01-25T02:47:39.1662430Z Jan 25 02:47:39 Checking for exceptions...
> 2023-01-25T02:47:39.2893767Z Jan 25 02:47:39 Found exception in log files;
> printing first 500 lines; see full logs for details:
> [...]
> 2023-01-25T02:47:39.5674568Z Jan 25 02:47:39 Checking for non-empty .out
> files...
> 2023-01-25T02:47:39.5675055Z Jan 25 02:47:39 No non-empty .out files.
> 2023-01-25T02:47:39.5675352Z Jan 25 02:47:39
> 2023-01-25T02:47:39.5676104Z Jan 25 02:47:39 [FAIL] 'RocksDB Memory
> Management end-to-end test' failed after 1 minutes and 50 seconds! Test
> exited with exit code 0 but the logs contained errors, exceptions or
> non-empty .out files
> {code}
> The only exception being reported in the Flink logs is due to a warning:
> {code}
> 2023-01-25 02:47:38,242 WARN
> org.apache.flink.runtime.checkpoint.CheckpointFailureManager [] - Failed to
> trigger or complete checkpoint 1 for job 421e4c00ef175b3b133d63cbfe9bca8b. (0
> consecutive failed attempts so far)
> org.apache.flink.runtime.checkpoint.CheckpointException: Checkpoint
> Coordinator is suspending.
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.stopCheckpointScheduler(CheckpointCoordinator.java:1970)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinatorDeActivator.jobStatusChanges(CheckpointCoordinatorDeActivator.java:46)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.notifyJobStatusChange(DefaultExecutionGraph.java:1578)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.transitionState(DefaultExecutionGraph.java:1173)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.transitionState(DefaultExecutionGraph.java:1145)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.executiongraph.DefaultExecutionGraph.cancel(DefaultExecutionGraph.java:973)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.scheduler.SchedulerBase.cancel(SchedulerBase.java:671)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.jobmaster.JobMaster.cancel(JobMaster.java:461)
> ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> ~[?:1.8.0_352]
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> ~[?:1.8.0_352]
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> ~[?:1.8.0_352]
> at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_352]
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$handleRpcInvocation$1(AkkaRpcActor.java:309)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.concurrent.akka.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:307)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:222)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:84)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:168)
> ~[flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:24)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:20)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at scala.PartialFunction.applyOrElse(PartialFunction.scala:127)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:20)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at
> scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.actor.Actor.aroundReceive(Actor.scala:537)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.actor.Actor.aroundReceive$(Actor.scala:535)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:220)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:579)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.actor.ActorCell.invoke(ActorCell.scala:547)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:270)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.dispatch.Mailbox.run(Mailbox.scala:231)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at akka.dispatch.Mailbox.exec(Mailbox.scala:243)
> [flink-rpc-akka_98d6268d-6cd0-412b-bd3c-ff411c887a5b.jar:1.17-SNAPSHOT]
> at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
> [?:1.8.0_352]
> at
> java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
> [?:1.8.0_352]
> at
> java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
> [?:1.8.0_352]
> at
> java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
> [?:1.8.0_352]
> {code}
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=45185&view=logs&j=bea52777-eaf8-5663-8482-18fbc3630e81&t=b2642e3a-5b86-574d-4c8a-f7e2842bfb14&l=5117
--
This message was sent by Atlassian Jira
(v8.20.10#820010)