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

Support disabling project rewrite when multiple exprs in the project reference the same sub project field.

    XMLWordPrintableJSON

Details

    Description

      When multiple top projects reference the same bottom project, project rewrite rules may result in complex projects being calculated multiple times.

      Take the following SQL as an example:

      create table test_source(a varchar) with ('connector'='datagen');
      
      explain plan for select a || 'a' as a, a || 'b' as b FROM (select REGEXP_REPLACE(a, 'aaa', 'bbb') as a FROM test_source);
      

      The final SQL plan is as follows:

      == Abstract Syntax Tree ==
      LogicalProject(a=[||($0, _UTF-16LE'a')], b=[||($0, _UTF-16LE'b')])
      +- LogicalProject(a=[REGEXP_REPLACE($0, _UTF-16LE'aaa', _UTF-16LE'bbb')])
         +- LogicalTableScan(table=[[default_catalog, default_database, test_source]])
      
      == Optimized Physical Plan ==
      Calc(select=[||(REGEXP_REPLACE(a, _UTF-16LE'aaa', _UTF-16LE'bbb'), _UTF-16LE'a') AS a, ||(REGEXP_REPLACE(a, _UTF-16LE'aaa', _UTF-16LE'bbb'), _UTF-16LE'b') AS b])
      +- TableSourceScan(table=[[default_catalog, default_database, test_source]], fields=[a])
      
      == Optimized Execution Plan ==
      Calc(select=[||(REGEXP_REPLACE(a, 'aaa', 'bbb'), 'a') AS a, ||(REGEXP_REPLACE(a, 'aaa', 'bbb'), 'b') AS b])
      +- TableSourceScan(table=[[default_catalog, default_database, test_source]], fields=[a])
      

      It can be observed that after project write, regex_place is calculated twice. Generally speaking, regular expression matching is a time-consuming operation and we usually do not want it to be calculated multiple times. Therefore, for this scenario, we can support disabling project rewrite.

      After disabling some rules, the final plan we obtained is as follows:

      == Abstract Syntax Tree ==
      LogicalProject(a=[||($0, _UTF-16LE'a')], b=[||($0, _UTF-16LE'b')])
      +- LogicalProject(a=[REGEXP_REPLACE($0, _UTF-16LE'aaa', _UTF-16LE'bbb')])
         +- LogicalTableScan(table=[[default_catalog, default_database, test_source]])
      
      == Optimized Physical Plan ==
      Calc(select=[||(a, _UTF-16LE'a') AS a, ||(a, _UTF-16LE'b') AS b])
      +- Calc(select=[REGEXP_REPLACE(a, _UTF-16LE'aaa', _UTF-16LE'bbb') AS a])
         +- TableSourceScan(table=[[default_catalog, default_database, test_source]], fields=[a])
      
      == Optimized Execution Plan ==
      Calc(select=[||(a, 'a') AS a, ||(a, 'b') AS b])
      +- Calc(select=[REGEXP_REPLACE(a, 'aaa', 'bbb') AS a])
         +- TableSourceScan(table=[[default_catalog, default_database, test_source]], fields=[a])
      

      After testing, we probably need to modify these few rules:

      org.apache.flink.table.planner.plan.rules.logical.FlinkProjectMergeRule

      org.apache.flink.table.planner.plan.rules.logical.FlinkCalcMergeRule

      org.apache.flink.table.planner.plan.rules.logical.FlinkProjectCalcMergeRule

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              hackergin Feng Jin
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated: