[ 
https://issues.apache.org/jira/browse/FLINK-28024?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17556788#comment-17556788
 ] 

Matthias Pohl edited comment on FLINK-28024 at 6/21/22 10:08 AM:
-----------------------------------------------------------------

It appears that the TaskManager failed fatally in this test run indicated by 
the following log lines:
{code:java}
16:28:08,303 [Cancellation Watchdog for Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] WARN  
org.apache.flink.runtime.taskmanager.Task                    [] - Task 'Source: 
Custom Source (2/4)#0' did not react to cancelling signal - n
otifying TM; it is stuck for 180 seconds in method:
 java.lang.Object.wait(Native Method)
java.lang.Thread.join(Thread.java:1252)
java.lang.Thread.join(Thread.java:1326)
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestExecutorImpl.close(ChannelStateWriteRequestExecutorImpl.java:166)
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl.close(ChannelStateWriterImpl.java:234)
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.cancel(SubtaskCheckpointCoordinatorImpl.java:560)
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.close(SubtaskCheckpointCoordinatorImpl.java:547)
org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$1236/820205024.close(Unknown
 Source)
org.apache.flink.util.IOUtils.closeAll(IOUtils.java:254)
org.apache.flink.core.fs.AutoCloseableRegistry.doClose(AutoCloseableRegistry.java:72)
org.apache.flink.util.AbstractAutoCloseableRegistry.close(AbstractAutoCloseableRegistry.java:127)
org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUp(StreamTask.java:938)
org.apache.flink.runtime.taskmanager.Task.lambda$restoreAndInvoke$1(Task.java:923)
org.apache.flink.runtime.taskmanager.Task$$Lambda$1951/1705350353.run(Unknown 
Source)
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:935)
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:923)
org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:728)
org.apache.flink.runtime.taskmanager.Task.run(Task.java:550)
java.lang.Thread.run(Thread.java:748)16:28:08,303 [Cancellation Watchdog for 
Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] ERROR 
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Task did not 
exit gracefully within 180 + seconds.
org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
within 180 + seconds.
        at 
org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1778)
 [flink-runtime-1.16-SNAPSHOT.jar:1.16-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_292]
16:28:08,303 [Cancellation Watchdog for Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] ERROR 
org.apache.flink.runtime.minicluster.MiniCluster             [] - TaskManager 
#0 failed.
org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
within 180 + seconds.
        at 
org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1778)
 [flink-runtime-1.16-SNAPSHOT.jar:1.16-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_292] {code}

The job is set to have a parallelism of 4 with two slots per TaskManager and 
two TaskManager being available initially. After one TM shutdown fatally, we 
end up with only two slots that never satisfy the 4 required slots:
{code}
16:30:01,416 [flink-akka.actor.default-dispatcher-38] WARN  
org.apache.flink.runtime.jobmaster.slotpool.DeclarativeSlotPoolBridge [] - 
Could not acquire the minimum required resources, failing slot requests. 
Acquired: 
[ResourceRequirement{resourceProfile=ResourceProfile{taskHeapMemory=512.0
00gb (549755813888 bytes), taskOffHeapMemory=512.000gb (549755813888 bytes), 
managedMemory=6.000mb (6291456 bytes), networkMemory=32.000mb (33554432 
bytes)}, numberOfRequiredSlots=2}]. Current slot pool status: Registered TMs: 
1, registered slots: 2 free slots: 0
16:30:01,422 [flink-akka.actor.default-dispatcher-38] INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (3/4) 
(b8f79df70c20ee5cc51bd99bd6852bb8_bc764cd8ddf7a0cff126f51c16239658_2_1) 
switched from SCHEDULED to FAILED on [unassigned resource].
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
Could not acquire the minimum required resources.
{code}

That's where we end up in the infinite loop of failover which spills the disk 
with logs.


was (Author: mapohl):
It appears that the TaskManager failed fatally in this test run indicated by 
the following log lines:
{code:java}
16:28:08,303 [Cancellation Watchdog for Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] WARN  
org.apache.flink.runtime.taskmanager.Task                    [] - Task 'Source: 
Custom Source (2/4)#0' did not react to cancelling signal - n
otifying TM; it is stuck for 180 seconds in method:
 java.lang.Object.wait(Native Method)
java.lang.Thread.join(Thread.java:1252)
java.lang.Thread.join(Thread.java:1326)
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestExecutorImpl.close(ChannelStateWriteRequestExecutorImpl.java:166)
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl.close(ChannelStateWriterImpl.java:234)
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.cancel(SubtaskCheckpointCoordinatorImpl.java:560)
org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.close(SubtaskCheckpointCoordinatorImpl.java:547)
org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$1236/820205024.close(Unknown
 Source)
org.apache.flink.util.IOUtils.closeAll(IOUtils.java:254)
org.apache.flink.core.fs.AutoCloseableRegistry.doClose(AutoCloseableRegistry.java:72)
org.apache.flink.util.AbstractAutoCloseableRegistry.close(AbstractAutoCloseableRegistry.java:127)
org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUp(StreamTask.java:938)
org.apache.flink.runtime.taskmanager.Task.lambda$restoreAndInvoke$1(Task.java:923)
org.apache.flink.runtime.taskmanager.Task$$Lambda$1951/1705350353.run(Unknown 
Source)
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:935)
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:923)
org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:728)
org.apache.flink.runtime.taskmanager.Task.run(Task.java:550)
java.lang.Thread.run(Thread.java:748)16:28:08,303 [Cancellation Watchdog for 
Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] ERROR 
org.apache.flink.runtime.taskexecutor.TaskExecutor           [] - Task did not 
exit gracefully within 180 + seconds.
org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
within 180 + seconds.
        at 
org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1778)
 [flink-runtime-1.16-SNAPSHOT.jar:1.16-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_292]
16:28:08,303 [Cancellation Watchdog for Source: Custom Source (2/4)#0 
(b8f79df70c20ee5cc51bd99bd6852bb8_feca28aff5a3958840bee985ee7de4d3_1_0).] ERROR 
org.apache.flink.runtime.minicluster.MiniCluster             [] - TaskManager 
#0 failed.
org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
within 180 + seconds.
        at 
org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1778)
 [flink-runtime-1.16-SNAPSHOT.jar:1.16-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_292] {code}

The job is set to have a parallelism of 4 with two slots per TaskManager and 
two TaskManager being available initially. After one TM shutdown fatally, we 
end up with only two slots that never satisfy the 4 required slots:
{code}
16:30:01,416 [flink-akka.actor.default-dispatcher-38] WARN  
org.apache.flink.runtime.jobmaster.slotpool.DeclarativeSlotPoolBridge [] - 
Could not acquire the minimum required resources, failing slot requests. 
Acquired: 
[ResourceRequirement{resourceProfile=ResourceProfile{taskHeapMemory=512.0
00gb (549755813888 bytes), taskOffHeapMemory=512.000gb (549755813888 bytes), 
managedMemory=6.000mb (6291456 bytes), networkMemory=32.000mb (33554432 
bytes)}, numberOfRequiredSlots=2}]. Current slot pool status: Registered TMs: 
1, registered slots: 2 free slots: 0
16:30:01,422 [flink-akka.actor.default-dispatcher-38] INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (3/4) 
(b8f79df70c20ee5cc51bd99bd6852bb8_bc764cd8ddf7a0cff126f51c16239658_2_1) 
switched from SCHEDULED to FAILED on [unassigned resource].
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: 
Could not acquire the minimum required resources.
{code}

That's where we end up in the infinite loop of failover.

> Azure worker stops operating due to log files becoming too big
> --------------------------------------------------------------
>
>                 Key: FLINK-28024
>                 URL: https://issues.apache.org/jira/browse/FLINK-28024
>             Project: Flink
>          Issue Type: Bug
>          Components: Build System / Azure Pipelines
>    Affects Versions: 1.16.0
>            Reporter: Matthias Pohl
>            Assignee: Matthias Pohl
>            Priority: Blocker
>              Labels: test-stability
>         Attachments: testWithRocksDbBackendIncremental.log.gz
>
>
> We observed several situations already where log files reached a file size of 
> over 120G. This caused the worker's disk usage to reach 100% resulting in the 
> worker machine to go "offline", i.e. not being available to pick up new tasks.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to