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

The concurrentExceptions doesn't work

    XMLWordPrintableJSON

Details

    Description

      First of all, thanks to mapohl for helping double-check in advance that this was indeed a bug .

      Displaying exception history in WebUI is supported in FLINK-6042.

      What's the concurrentExceptions?

      When an execution fails due to an exception, other executions in the same region will also restart, and the first Exception is rootException. If other restarted executions also report Exception at this time, we hope to collect these exceptions and Displayed to the user as concurrentExceptions.

      What's this bug?

      The concurrentExceptions is always empty in production, even if other executions report exception at very close times.

      Why doesn't it work?

      If one job has all-to-all shuffle, this job only has one region, and this region has a lot of executions. If one execution throw exception:

      • JobMaster will mark the state as FAILED for this execution.
      • The rest of executions of this region will be marked to CANCELING.
        • This call stack can be found at FLIP-364 part-4.2.3 

      When these executions throw exception as well, it JobMaster will mark the state from CANCELING to CANCELED instead of FAILED.

      The CANCELED execution won't call FAILED logic, so their exceptions are ignored.

      Note: all reports are executed inside of JobMaster RPC thread, it's single thread. So these reports are executed serially. So only one execution is marked to FAILED, and the rest of executions will be marked to CANCELED later.

      How to fix it?

      Offline discuss with mapohl , we need to discuss with community should we keep the concurrentExceptions first.

      • If no, we can remove related logic directly
      • If yew, we discuss how to fix it later.

      Attachments

        1. screenshot-1.png
          364 kB
          Rui Fan

        Issue Links

          Activity

            People

              fanrui Rui Fan
              fanrui Rui Fan
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: