Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-12106

Jobmanager is killing FINISHED taskmanger containers, causing exception in still running Taskmanagers an

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Duplicate
    • 1.7.2
    • None
    • Deployment / YARN
    • None
    • Hadoop:  hdp/2.5.6.0-40

      Flink: 2.7.2

    Description

      When running a single flink job on YARN, some of the taskmanger containers reach the FINISHED state before others.  It appears that, after receiving final execution state FINISHED from a taskmanager, jobmanager is waiting ~68 seconds and then freeing the associated slot in the taskmanager.  After and additional 60 seconds, jobmanager is stopping the same taskmanger because TaskExecutor exceeded the idle timeout.

      Meanwhile, other taskmangers are still working to complete the job.  Within 10 seconds after the taskmanger container above is stopped, the remaining task managers receive an exception due to loss of connection to the stopped taskmanager.  These exceptions result job failure.

       

      Relevant logs:

      2019-04-03 13:49:00,013 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering TaskManager with ResourceID container_1553017480503_0158_01_000038 (akka.tcp://flink@hadoop4:42745/user/taskmanager_0) at ResourceManager

      2019-04-03 13:49:05,900 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering TaskManager with ResourceID container_1553017480503_0158_01_000059 (akka.tcp://flink@hadoop9:55042/user/taskmanager_0) at ResourceManager

       

       

      2019-04-03 13:48:51,132 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_1553017480503_0158_01_000077 - Remaining pending container requests: 6

      2019-04-03 13:48:52,862 INFO  org.apache.flink.yarn.YarnTaskExecutorRunner                  -     -Dlog.file=/hadoop/yarn/log/application_1553017480503_0158/container_1553017480503_0158_01_000077/taskmanager.log

      2019-04-03 13:48:57,490 INFO  org.apache.flink.runtime.io.network.netty.NettyServer         - Successful initialization (took 202 ms). Listening on SocketAddress /192.168.230.69:40140.

      2019-04-03 13:49:12,575 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering TaskManager with ResourceID container_1553017480503_0158_01_000077 (akka.tcp://flink@hadoop9:51525/user/taskmanager_0) at ResourceManager

      2019-04-03 13:49:12,631 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for AllocationID{42fed3e5a136240c23cc7b394e3249e9}.

      2019-04-03 14:58:15,188 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Un-registering task and sending final execution state FINISHED to JobManager for task DataSink (com.anovadata.alexflinklib.sinks.bucketing.BucketingOutputFormat@26874f2c) a4b5fb32830d4561147b2714828109e2.

      2019-04-03 14:59:23,049 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool          - Releasing idle slot [AllocationID\{42fed3e5a136240c23cc7b394e3249e9}].

      2019-04-03 14:59:23,058 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable      - Free slot TaskSlot(index:0, state:ACTIVE, resource profile: ResourceProfile{cpuCores=1.7976931348623157E308, heapMemoryInMB=2147483647, directMemoryInMB=2147483647, nativeMemoryInMB=2147483647, networkMemoryInMB=2147483647}, allocationId: AllocationID{42fed3e5a136240c23cc7b394e3249e9}, jobId: a6c4e367698c15cdf168d19a89faff1d).

      2019-04-03 15:00:02,641 INFO  org.apache.flink.yarn.YarnResourceManager                     - Stopping container container_1553017480503_0158_01_000077.

      2019-04-03 15:00:02,646 INFO  org.apache.flink.yarn.YarnResourceManager                     - Closing TaskExecutor connection container_1553017480503_0158_01_000077 because: TaskExecutor exceeded the idle timeout.

       

       

      2019-04-03 13:48:48,902 INFO  org.apache.flink.yarn.YarnTaskExecutorRunner                  -     -Dlog.file=/data1/hadoop/yarn/log/application_1553017480503_0158/container_1553017480503_0158_01_000059/taskmanager.log

      2019-04-03 14:59:24,677 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - Flushing mem columnStore to file. allocated memory: 109479981

      2019-04-03 15:00:05,696 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - mem size 135014409 > 134217728: flushing 1930100 records to disk.

      2019-04-03 15:00:05,696 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - Flushing mem columnStore to file. allocated memory: 102677684

      2019-04-03 15:00:08,671 ERROR org.apache.flink.runtime.operators.BatchTask                  - Error in task code:  CHAIN Partition -> FlatMap 

      org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: Lost connection to task manager 'hadoop9/192.168.230.69:40140'. This indicates that the remote task manager was lost.

      2019-04-03 15:00:08,714 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Un-registering task and sending final execution state FAILED to JobManager for task CHAIN Partition -> FlatMap

      2019-04-03 15:00:08,812 INFO  org.apache.flink.runtime.taskmanager.Task                     - Attempting to cancel task DataSink ()

      2019-04-03 15:00:08,812 INFO  org.apache.flink.runtime.taskmanager.Task                     - DataSink () switched from RUNNING to CANCELING.

      2019-04-03 15:00:08,812 INFO  org.apache.flink.runtime.taskmanager.Task                     - Triggering cancellation of task code DataSink ()

       

       

      2019-04-03 13:48:44,562 INFO  org.apache.flink.yarn.YarnTaskExecutorRunner                  -     -Dlog.file=/data8/hadoop/yarn/log/application_1553017480503_0158/container_1553017480503_0158_01_000038/taskmanager.log

      2019-04-03 14:59:18,620 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - Flushing mem columnStore to file. allocated memory: 0

      2019-04-03 14:59:48,088 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - mem size 136179972 > 134217728: flushing 1930100 records to disk.

      2019-04-03 14:59:48,088 INFO  org.apache.parquet.hadoop.InternalParquetRecordWriter         - Flushing mem columnStore to file. allocated memory: 103333893

      2019-04-03 15:00:08,692 ERROR org.apache.flink.runtime.operators.BatchTask                  - Error in task code:  CHAIN Partition -> FlatMap

      org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException: Lost connection to task manager 'hadoop9/192.168.230.69:40140'. This indicates that the remote task manager was lost.

      2019-04-03 15:00:08,741 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Un-registering task and sending final execution state FAILED to JobManager for task CHAIN Partition -> FlatMap

      2019-04-03 15:00:08,817 INFO  org.apache.flink.runtime.taskmanager.Task                     - Attempting to cancel task DataSink ()

      2019-04-03 15:00:08,817 INFO  org.apache.flink.runtime.taskmanager.Task                     - DataSink () switched from RUNNING to CANCELING.

      2019-04-03 15:00:08,817 INFO  org.apache.flink.runtime.taskmanager.Task                     - Triggering cancellation of task code DataSink ()

       

       

      2019-04-03 15:00:09,196 INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher            - Job a6c4e367698c15cdf168d19a89faff1d reached globally terminal state FAILED.

       

        

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              Hutchins John
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: