Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-30531 Reduce operator chain call stack depth
  3. FLINK-30536

Remove CountingOutput from per-record code path for most operators

    XMLWordPrintableJSON

Details

    Description

      For the example program shown below, a CountingOutput will be added to the per-record code path for each map operation added in the program. This reduces the Flink performance as it increases the function call stack depth on the critical code path.

      DataStream<Long> stream = env.fromSequence(1, 500000000L);
      for (int i = 0; i < 10; i++) {
          stream = stream.map(x -> x);
      }
      stream.addSink(new DiscardingSink<>());
      

       

      Instead of adding a CountingOutput that wraps around ChainingOutput, we can add a Counter in the ChainingOutput to achieve the same goal in most cases. We can do this when constructing the operator chain.

      By making the change described above, we can reduce the call stack depth, increase the chance for function inline, and reduce the Flink runtime overhead.

       

      Prior to the proposed change, each map() operation in the above program would add the following 3 functions on the call stack needed to produce a record:

      • CountingOutput#collect
      • ChainingOutput#collect
      • StreamMap#processElement

      After the proposed change, the number of functions added for each map() operation would be reduced from 3 to 2, with ChainingOutput#collect removed from the call stack.

       

      Here are the benchmark results obtained by running the above program with parallelism=1 and object re-use enabled. The results are averaged across 5 runs for each setup.

      • Prior to the proposed change, the average execution time is 56.54 sec with std=4.9 sec.
      • After the proposed change, the average execution time is 63.43 sec with std=6.3 sec.
      • The proposed change increases throughput by 12.2%.

       

      Attachments

        Issue Links

          Activity

            People

              lindong Dong Lin
              lindong Dong Lin
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: