Uploaded image for project: 'Apache Tez'
  1. Apache Tez
  2. TEZ-3976

Batch ShuffleManager error report events

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.10.0
    • None
    • None

    Description

      The symptoms are a lot of these logs are being shown:

      2018-06-15T18:09:35,811 INFO  [Fetcher_B {Reducer_5} #0 ()] org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager: Reducer_5: Fetch failed for src: InputAttemptIdentifier [inputIdentifier=701, attemptNumber=0, pathComponent=attempt_1529044441963_0021_34_01_000701_0_12541_0, spillType=2, spillId=0]InputIdentifier: InputAttemptIdentifier [inputIdentifier=701, attemptNumber=0, pathComponent=attempt_1529044441963_0021_34_01_000701_0_12541_0, spillType=2, spillId=0], connectFailed: true
      2018-06-15T18:09:35,811 WARN  [Fetcher_B {Reducer_5} #1 ()] org.apache.tez.runtime.library.common.shuffle.Fetcher: copyInputs failed for tasks [InputAttemptIdentifier [inputIdentifier=589, attemptNumber=0, pathComponent=attempt_1529044441963_0021_34_01_000589_0_12445_0, spillType=2, spillId=0]]
      2018-06-15T18:09:35,811 INFO  [Fetcher_B {Reducer_5} #1 ()] org.apache.tez.runtime.library.common.shuffle.impl.ShuffleManager: Reducer_5: Fetch failed for src: InputAttemptIdentifier [inputIdentifier=589, attemptNumber=0, pathComponent=attempt_1529044441963_0021_34_01_000589_0_12445_0, spillType=2, spillId=0]InputIdentifier: InputAttemptIdentifier [inputIdentifier=589, attemptNumber=0, pathComponent=attempt_1529044441963_0021_34_01_000589_0_12445_0, spillType=2, spillId=0], connectFailed: true
      

      Each of those translate into an event in the AM which finally crashes due to OOM after around 30 minutes and around 10 million shuffle input errors (and 10 million lines like the previous ones). When the ShufflerManager is closed and the counters reported there are many shuffle input errors, some of those logs are:

      2018-06-15T17:46:30,988  INFO [TezTR-441963_21_34_4_0_4 (1529044441963_0021_34_04_000000_4)] runtime.LogicalIOProcessorRuntimeTask: Final Counters for attempt_1529044441963_0021_34_04_000000_4: Counters: 43 [[org.apache.tez.common.counters.TaskCounter SPILLED_RECORDS=0, NUM_SHUFFLED_INPUTS=26, NUM_FAILED_SHUFFLE_INPUTS=858965, INPUT_RECORDS_PROCESSED=26, OUTPUT_RECORDS=1, OUTPUT_LARGE_RECORDS=0, OUTPUT_BYTES=779472, OUTPUT_BYTES_WITH_OVERHEAD=779483, OUTPUT_BYTES_PHYSICAL=780146, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILL_COUNT=0, SHUFFLE_BYTES=4207563, SHUFFLE_BYTES_DECOMPRESSED=20266603, SHUFFLE_BYTES_TO_MEM=3380616, SHUFFLE_BYTES_TO_DISK=0, SHUFFLE_BYTES_DISK_DIRECT=826947, SHUFFLE_PHASE_TIME=52516, FIRST_EVENT_RECEIVED=1, LAST_EVENT_RECEIVED=1185][HIVE RECORDS_OUT_INTERMEDIATE_^[[1;35;40m^[[KReducer_12^[[m^[[K=1, RECORDS_OUT_OPERATOR_GBY_159=1, RECORDS_OUT_OPERATOR_RS_160=1][TaskCounter_^[[1;35;40m^[[KReducer_12^[[m^[[K_INPUT_Map_11 FIRST_EVENT_RECEIVED=1, INPUT_RECORDS_PROCESSED=26, LAST_EVENT_RECEIVED=1185, NUM_FAILED_SHUFFLE_INPUTS=858965, NUM_SHUFFLED_INPUTS=26, SHUFFLE_BYTES=4207563, SHUFFLE_BYTES_DECOMPRESSED=20266603, SHUFFLE_BYTES_DISK_DIRECT=826947, SHUFFLE_BYTES_TO_DISK=0, SHUFFLE_BYTES_TO_MEM=3380616, SHUFFLE_PHASE_TIME=52516][TaskCounter_^[[1;35;40m^[[KReducer_12^[[m^[[K_OUTPUT_Map_1 ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILL_COUNT=0, OUTPUT_BYTES=779472, OUTPUT_BYTES_PHYSICAL=780146, OUTPUT_BYTES_WITH_OVERHEAD=779483, OUTPUT_LARGE_RECORDS=0, OUTPUT_RECORDS=1, SPILLED_RECORDS=0]]
      
      
      2018-06-15T17:46:32,271 INFO  [TezTR-441963_21_34_3_15_1 ()] org.apache.tez.runtime.LogicalIOProcessorRuntimeTask: Final Counters for attempt_1529044441963_0021_34_03_000015_1: Counters: 87 [[File System Counters FILE_BYTES_READ=0, FILE_BYTES_WRITTEN=0, FILE_READ_OPS=0, FILE_LARGE_READ_OPS=0, FILE_WRITE_OPS=0, HDFS_BYTES_READ=2344929, HDFS_BYTES_WRITTEN=0, HDFS_READ_OPS=5, HDFS_LARGE_READ_OPS=0, HDFS_WRITE_OPS=0][org.apache.tez.common.counters.TaskCounter SPILLED_RECORDS=0, NUM_SHUFFLED_INPUTS=1, NUM_FAILED_SHUFFLE_INPUTS=105195, INPUT_RECORDS_PROCESSED=397, INPUT_SPLIT_LENGTH_BYTES=21563271, OUTPUT_RECORDS=15737, OUTPUT_LARGE_RECORDS=0, OUTPUT_BYTES=1235818, OUTPUT_BYTES_WITH_OVERHEAD=1267307, OUTPUT_BYTES_PHYSICAL=357520, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILL_COUNT=0, SHUFFLE_BYTES=31, SHUFFLE_BYTES_DECOMPRESSED=17, SHUFFLE_BYTES_TO_MEM=31, SHUFFLE_BYTES_TO_DISK=0, SHUFFLE_BYTES_DISK_DIRECT=0, SHUFFLE_PHASE_TIME=50525, FIRST_EVENT_RECEIVED=9, LAST_EVENT_RECEIVED=61][HIVE DESERIALIZE_ERRORS=0, RECORDS_IN_Map_11=395611, RECORDS_OUT_INTERMEDIATE_Map_11=15737, RECORDS_OUT_OPERATOR_FIL_152=395611, RECORDS_OUT_OPERATOR_GBY_157=1, RECORDS_OUT_OPERATOR_MAPJOIN_154=15736, RECORDS_OUT_OPERATOR_MAP_0=0, RECORDS_OUT_OPERATOR_RS_155=15736, RECORDS_OUT_OPERATOR_RS_158=1, RECORDS_OUT_OPERATOR_SEL_153=395611, RECORDS_OUT_OPERATOR_SEL_156=15736, RECORDS_OUT_OPERATOR_TS_26=395611][TaskCounter_Map_11_INPUT_Map_13 FIRST_EVENT_RECEIVED=9, INPUT_RECORDS_PROCESSED=1, LAST_EVENT_RECEIVED=61, NUM_FAILED_SHUFFLE_INPUTS=105195, NUM_SHUFFLED_INPUTS=1, SHUFFLE_BYTES=31, SHUFFLE_BYTES_DECOMPRESSED=17, SHUFFLE_BYTES_DISK_DIRECT=0, SHUFFLE_BYTES_TO_DISK=0, SHUFFLE_BYTES_TO_MEM=31, SHUFFLE_PHASE_TIME=50525][TaskCounter_Map_11_INPUT_supplier INPUT_RECORDS_PROCESSED=396, INPUT_SPLIT_LENGTH_BYTES=21563271][TaskCounter_Map_11_OUTPUT_Reducer_12 ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILL_COUNT=0, OUTPUT_BYTES=779474, OUTPUT_BYTES_PHYSICAL=164787, OUTPUT_BYTES_WITH_OVERHEAD=779485, OUTPUT_LARGE_RECORDS=0, OUTPUT_RECORDS=1, SPILLED_RECORDS=0][TaskCounter_Map_11_OUTPUT_Reducer_6 ADDITIONAL_SPILLS_BYTES_READ=0, ADDITIONAL_SPILLS_BYTES_WRITTEN=0, ADDITIONAL_SPILL_COUNT=0, OUTPUT_BYTES=456344, OUTPUT_BYTES_PHYSICAL=192733, OUTPUT_BYTES_WITH_OVERHEAD=487822, OUTPUT_LARGE_RECORDS=0, OUTPUT_RECORDS=15736, SPILLED_RECORDS=0][org.apache.hadoop.hive.llap.counters.LlapIOCounters ALLOCATED_BYTES=9633792, ALLOCATED_USED_BYTES=7976706, CACHE_HIT_BYTES=0, CACHE_MISS_BYTES=2344364, CONSUMER_TIME_NS=1136392475, DECODE_TIME_NS=140377915, HDFS_TIME_NS=145825282, METADATA_CACHE_MISS=4, NUM_DECODED_BATCHES=41, NUM_VECTOR_BATCHES=396, ROWS_EMITTED=395611, SELECTED_ROWGROUPS=41, TOTAL_IO_TIME_NS=1234990631]]
      

      I think this is happening because the fetcher is in this loop which is started here and not stopped until the processor is closed (which is called from here or here).

      In ShuffleManager we seem to keep track of TaskCounter.NUM_FAILED_SHUFFLE_INPUTS but nothing is done when the value gets too high. Maybe something similar to ShuffleScheduler should be done where the fetchers are only retried a certain amount of times.

      Attachments

        1. TEZ-3976.1.patch
          17 kB
          Jaume M
        2. TEZ-3976.2.patch
          19 kB
          Jaume M
        3. TEZ-3976.3.patch
          17 kB
          Jaume M
        4. TEZ-3976.4.patch
          16 kB
          Jaume M
        5. TEZ-3976.5.patch
          16 kB
          Jaume M
        6. TEZ-3976.6.patch
          17 kB
          Jaume M
        7. TEZ-3976.7.patch
          18 kB
          Jaume M
        8. TEZ-3976.8.patch
          0.8 kB
          Jaume M
        9. TEZ-3976.9.patch
          18 kB
          Jaume M

        Activity

          People

            jmarhuen Jaume M
            jmarhuen Jaume M
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: