[
https://issues.apache.org/jira/browse/FLINK-33186?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17773565#comment-17773565
]
Jiang Xin commented on FLINK-33186:
-----------------------------------
[~Sergey Nuyanzin] I don't think the failure is related to FLINK-28386 or
FLINK-32996.
The test code triggered a savepoint after some subtasks were completed, but
from the logs, it can be seen that the subtask `passA ->Sink: sinkA (2/4)` went
to the finished state right after initiating the savepoint. Then, TaskExecutor
believed that it had received a checkpoint request for an unknown task, thus it
failed. So I think it is an existing concurrency issue.
```
01:23:40,823 [flink-pekko.actor.default-dispatcher-8] INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph [] - passA ->
Sink: sinkA (1/4)
(3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_0_0) switc
hed from RUNNING to FINISHED.
01:23:40,823 [flink-pekko.actor.default-dispatcher-8] INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph [] - passA ->
Sink: sinkA (3/4)
(3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_2_0) switc
hed from RUNNING to FINISHED.
01:23:40,826 [flink-pekko.actor.default-dispatcher-4] INFO
org.apache.flink.runtime.jobmaster.JobMaster [] - Triggering
cancel-with-savepoint for job c82d241f9952e043dfed65318f0d962a.
01:23:40,828 [passA -> Sink: sinkA (2/4)#0] INFO
org.apache.flink.runtime.taskmanager.Task [] - passA ->
Sink: sinkA (2/4)#0
(3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0)
switched from RUNNING to FINISHED.
01:23:40,828 [passA -> Sink: sinkA (2/4)#0] INFO
org.apache.flink.runtime.taskmanager.Task [] - Freeing task
resources for passA -> Sink: sinkA (2/4)#0
(3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0).
01:23:40,828 [flink-pekko.actor.default-dispatcher-8] INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor [] -
Un-registering task and sending final execution state FINISHED to JobManager
for task passA -> Sink: sinkA (2/4)#0
3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0.
01:23:40,829 [flink-pekko.actor.default-dispatcher-4] INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph [] - passA ->
Sink: sinkA (2/4)
(3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0)
switched from RUNNING to FINISHED.
01:23:40,829 [ Checkpoint Timer] INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Triggering
checkpoint 2 (type=SavepointType\{name='Savepoint', postCheckpointAction=NONE,
formatType=CANONICAL}) @ 1695864220828 for job c82d241f9952e043dfed65318f0d962a.
01:23:40,917 [flink-pekko.actor.default-dispatcher-8] INFO
org.apache.flink.runtime.checkpoint.CheckpointCoordinator [] - Triggering
Checkpoint 2 for job c82d241f9952e043dfed65318f0d962a failed due to
org.apache.flink.runtime.checkpoint.CheckpointException: TaskManager received a
checkpoint request for unknown task
3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0. Failure
reason: Task local checkpoint failure.
01:23:40,918 [ Checkpoint Timer] WARN
org.apache.flink.runtime.checkpoint.CheckpointFailureManager [] - Failed to
trigger or complete checkpoint 2 for job c82d241f9952e043dfed65318f0d962a. (0
consecutive failed attempts so far)
org.apache.flink.runtime.checkpoint.CheckpointException: TaskManager received a
checkpoint request for unknown task
3146192e86ef62554451af0e39df80b5_51397532e2d9c7a21097a30d590b3114_1_0. Failure
reason: Task local checkpoint failure.
at
org.apache.flink.runtime.taskexecutor.TaskExecutor.triggerCheckpoint(TaskExecutor.java:1046)
~[flink-runtime-1.19-SNAPSHOT.jar:1.19-SNAPSHOT]
at sun.reflect.GeneratedMethodAccessor24.invoke(Unknown Source) ~[?:?]
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[?:1.8.0_292]
at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_292]
at
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.lambda$handleRpcInvocation$1(PekkoRpcActor.java:309)
~[?:?]
at
org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
~[flink-rpc-core-1.19-SNAPSHOT.jar:1.19-SNAPSHOT]
at
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcInvocation(PekkoRpcActor.java:307)
~[?:?]
at
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcMessage(PekkoRpcActor.java:222)
~[?:?]
at
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleMessage(PekkoRpcActor.java:168)
~[?:?]
at
org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:33) ~[?:?]
at
org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:29) ~[?:?]
at scala.PartialFunction.applyOrElse(PartialFunction.scala:127)
~[scala-library-2.12.7.jar:?]
at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126)
~[scala-library-2.12.7.jar:?]
at
org.apache.pekko.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:29)
~[?:?]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175)
~[scala-library-2.12.7.jar:?]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
~[scala-library-2.12.7.jar:?]
at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
~[scala-library-2.12.7.jar:?]
at org.apache.pekko.actor.Actor.aroundReceive(Actor.scala:547) ~[?:?]
at org.apache.pekko.actor.Actor.aroundReceive$(Actor.scala:545) ~[?:?]
at
org.apache.pekko.actor.AbstractActor.aroundReceive(AbstractActor.scala:229)
~[?:?]
at org.apache.pekko.actor.ActorCell.receiveMessage(ActorCell.scala:590)
~[?:?]
at org.apache.pekko.actor.ActorCell.invoke(ActorCell.scala:557) ~[?:?]
at org.apache.pekko.dispatch.Mailbox.processMailbox(Mailbox.scala:280)
~[?:?]
at org.apache.pekko.dispatch.Mailbox.run(Mailbox.scala:241) ~[?:?]
at org.apache.pekko.dispatch.Mailbox.exec(Mailbox.scala:253) ~[?:?]
at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289)
~[?:1.8.0_292]
at
java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056)
~[?:1.8.0_292]
at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692)
~[?:1.8.0_292]
at
java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:175)
~[?:1.8.0_292]
01:23:40,929 [ForkJoinPool-1-worker-25] INFO
org.apache.flink.runtime.minicluster.MiniCluster [] - Shutting down
Flink Mini Cluster
```
> CheckpointAfterAllTasksFinishedITCase.testRestoreAfterSomeTasksFinished
> fails on AZP
> -------------------------------------------------------------------------------------
>
> Key: FLINK-33186
> URL: https://issues.apache.org/jira/browse/FLINK-33186
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing
> Affects Versions: 1.19.0
> Reporter: Sergey Nuyanzin
> Assignee: Jiang Xin
> Priority: Critical
> Labels: test-stability
>
> This build
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=53509&view=logs&j=baf26b34-3c6a-54e8-f93f-cf269b32f802&t=8c9d126d-57d2-5a9e-a8c8-ff53f7b35cd9&l=8762
> fails as
> {noformat}
> Sep 28 01:23:43 Caused by:
> org.apache.flink.runtime.checkpoint.CheckpointException: Task local
> checkpoint failure.
> Sep 28 01:23:43 at
> org.apache.flink.runtime.checkpoint.PendingCheckpoint.abort(PendingCheckpoint.java:550)
> Sep 28 01:23:43 at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:2248)
> Sep 28 01:23:43 at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.abortPendingCheckpoint(CheckpointCoordinator.java:2235)
> Sep 28 01:23:43 at
> org.apache.flink.runtime.checkpoint.CheckpointCoordinator.lambda$null$9(CheckpointCoordinator.java:817)
> Sep 28 01:23:43 at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> Sep 28 01:23:43 at
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> Sep 28 01:23:43 at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> Sep 28 01:23:43 at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> Sep 28 01:23:43 at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> Sep 28 01:23:43 at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> Sep 28 01:23:43 at java.lang.Thread.run(Thread.java:748)
> {noformat}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)