This is an automated email from the ASF dual-hosted git repository.

snuyanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 4d762c2bdc0 [FLINK-34162][table] Migrate LogicalUnnestRule to java
4d762c2bdc0 is described below

commit 4d762c2bdc0720e1bf2615e3d30a5741c4688212
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Mon Apr 8 11:07:28 2024 +0200

    [FLINK-34162][table] Migrate LogicalUnnestRule to java
---
 .../plan/rules/logical/LogicalUnnestRule.java      | 180 +++++++++++++++++++++
 .../plan/rules/logical/LogicalUnnestRule.scala     | 133 ---------------
 2 files changed, 180 insertions(+), 133 deletions(-)

diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
new file mode 100644
index 00000000000..ffa884eed5f
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.rules.logical;
+
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableList;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.immutables.value.Value;
+
+import java.util.Collections;
+import java.util.Map;
+
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType;
+
+/**
+ * Planner rule that rewrites UNNEST to explode function.
+ *
+ * <p>Note: This class can only be used in HepPlanner.
+ */
[email protected]
+public class LogicalUnnestRule extends 
RelRule<LogicalUnnestRule.LogicalUnnestRuleConfig> {
+
+    public static final LogicalUnnestRule INSTANCE = 
LogicalUnnestRuleConfig.DEFAULT.toRule();
+
+    public LogicalUnnestRule(LogicalUnnestRule.LogicalUnnestRuleConfig config) 
{
+        super(config);
+    }
+
+    public boolean matches(RelOptRuleCall call) {
+        LogicalCorrelate join = call.rel(0);
+        RelNode right = getRel(join.getRight());
+        if (right instanceof LogicalFilter) {
+            LogicalFilter logicalFilter = (LogicalFilter) right;
+            RelNode relNode = getRel(logicalFilter.getInput());
+            if (relNode instanceof Uncollect) {
+                return !((Uncollect) relNode).withOrdinality;
+            } else if (relNode instanceof LogicalProject) {
+                LogicalProject logicalProject = (LogicalProject) relNode;
+                relNode = getRel(logicalProject.getInput());
+                if (relNode instanceof Uncollect) {
+                    return !((Uncollect) relNode).withOrdinality;
+                }
+                return false;
+            }
+        } else if (right instanceof LogicalProject) {
+            LogicalProject logicalProject = (LogicalProject) right;
+            RelNode relNode = getRel(logicalProject.getInput());
+            if (relNode instanceof Uncollect) {
+                Uncollect uncollect = (Uncollect) relNode;
+                return !uncollect.withOrdinality;
+            }
+            return false;
+        } else if (right instanceof Uncollect) {
+            Uncollect uncollect = (Uncollect) right;
+            return !uncollect.withOrdinality;
+        }
+        return false;
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+        LogicalCorrelate correlate = call.rel(0);
+        RelNode outer = getRel(correlate.getLeft());
+        RelNode array = getRel(correlate.getRight());
+
+        // convert unnest into table function scan
+        RelNode tableFunctionScan = convert(array, correlate);
+        // create correlate with table function scan as input
+        Correlate newCorrelate =
+                correlate.copy(correlate.getTraitSet(), 
ImmutableList.of(outer, tableFunctionScan));
+        call.transformTo(newCorrelate);
+    }
+
+    private RelNode convert(RelNode relNode, LogicalCorrelate correlate) {
+        if (relNode instanceof HepRelVertex) {
+            HepRelVertex hepRelVertex = (HepRelVertex) relNode;
+            relNode = convert(getRel(hepRelVertex), correlate);
+        }
+        if (relNode instanceof LogicalProject) {
+            LogicalProject logicalProject = (LogicalProject) relNode;
+            return logicalProject.copy(
+                    logicalProject.getTraitSet(),
+                    
ImmutableList.of(convert(getRel(logicalProject.getInput()), correlate)));
+        }
+        if (relNode instanceof LogicalFilter) {
+            LogicalFilter logicalFilter = (LogicalFilter) relNode;
+            return logicalFilter.copy(
+                    logicalFilter.getTraitSet(),
+                    ImmutableList.of(convert(getRel(logicalFilter.getInput()), 
correlate)));
+        }
+        if (relNode instanceof Uncollect) {
+            Uncollect uncollect = (Uncollect) relNode;
+            RelOptCluster cluster = correlate.getCluster();
+            FlinkTypeFactory typeFactory = 
ShortcutUtils.unwrapTypeFactory(cluster);
+            RelDataType relDataType =
+                    (RelDataType)
+                            ((Map.Entry) 
uncollect.getInput().getRowType().getFieldList().get(0))
+                                    .getValue();
+            LogicalType logicalType = 
FlinkTypeFactory.toLogicalType(relDataType);
+            BridgingSqlFunction sqlFunction =
+                    BridgingSqlFunction.of(
+                            cluster, 
BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS);
+            RexNode rexCall =
+                    cluster.getRexBuilder()
+                            .makeCall(
+                                    typeFactory.createFieldTypeFromLogicalType(
+                                            toRowType(
+                                                    
UnnestRowsFunction.getUnnestedType(
+                                                            logicalType))),
+                                    sqlFunction,
+                                    ((LogicalProject) 
getRel(uncollect.getInput())).getProjects());
+            return new LogicalTableFunctionScan(
+                    cluster,
+                    correlate.getTraitSet(),
+                    Collections.emptyList(),
+                    rexCall,
+                    null,
+                    rexCall.getType(),
+                    null);
+        } else {
+            throw new IllegalArgumentException("Unexpected input: " + relNode);
+        }
+    }
+
+    private RelNode getRel(RelNode rel) {
+        if (rel instanceof HepRelVertex) {
+            return ((HepRelVertex) rel).getCurrentRel();
+        }
+        return rel;
+    }
+
+    /** Rule configuration. */
+    @Value.Immutable(singleton = false)
+    public interface LogicalUnnestRuleConfig extends RelRule.Config {
+        LogicalUnnestRuleConfig DEFAULT =
+                ImmutableLogicalUnnestRule.LogicalUnnestRuleConfig.builder()
+                        .build()
+                        .withOperandSupplier(b0 -> 
b0.operand(LogicalCorrelate.class).anyInputs())
+                        .withDescription("LogicalUnnestRule");
+
+        @Override
+        default LogicalUnnestRule toRule() {
+            return new LogicalUnnestRule(this);
+        }
+    }
+}
diff --git 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala
 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala
deleted file mode 100644
index ed096096e58..00000000000
--- 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.table.planner.plan.rules.logical
-
-import org.apache.flink.table.functions.BuiltInFunctionDefinitions
-import org.apache.flink.table.planner.calcite.FlinkTypeFactory
-import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction
-import org.apache.flink.table.planner.utils.ShortcutUtils
-import org.apache.flink.table.runtime.functions.table.UnnestRowsFunction
-import org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toRowType
-
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptRuleOperand}
-import org.apache.calcite.plan.RelOptRule._
-import org.apache.calcite.plan.hep.HepRelVertex
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.core.Uncollect
-import org.apache.calcite.rel.logical._
-
-import java.util.Collections
-
-/**
- * Planner rule that rewrites UNNEST to explode function.
- *
- * Note: This class can only be used in HepPlanner.
- */
-class LogicalUnnestRule(operand: RelOptRuleOperand, description: String)
-  extends RelOptRule(operand, description) {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val join: LogicalCorrelate = call.rel(0)
-    val right = getRel(join.getRight)
-
-    right match {
-      // a filter is pushed above the table function
-      case filter: LogicalFilter =>
-        getRel(filter.getInput) match {
-          case u: Uncollect => !u.withOrdinality
-          case p: LogicalProject =>
-            getRel(p.getInput) match {
-              case u: Uncollect => !u.withOrdinality
-              case _ => false
-            }
-          case _ => false
-        }
-      case project: LogicalProject =>
-        getRel(project.getInput) match {
-          case u: Uncollect => !u.withOrdinality
-          case _ => false
-        }
-      case u: Uncollect => !u.withOrdinality
-      case _ => false
-    }
-  }
-
-  override def onMatch(call: RelOptRuleCall): Unit = {
-    val correlate: LogicalCorrelate = call.rel(0)
-    val outer = getRel(correlate.getLeft)
-    val array = getRel(correlate.getRight)
-
-    def convert(relNode: RelNode): RelNode = {
-      relNode match {
-        case rs: HepRelVertex =>
-          convert(getRel(rs))
-
-        case f: LogicalProject =>
-          f.copy(f.getTraitSet, ImmutableList.of(convert(getRel(f.getInput))))
-
-        case f: LogicalFilter =>
-          f.copy(f.getTraitSet, ImmutableList.of(convert(getRel(f.getInput))))
-
-        case uc: Uncollect =>
-          // convert Uncollect into TableFunctionScan
-          val cluster = correlate.getCluster
-          val typeFactory = ShortcutUtils.unwrapTypeFactory(cluster)
-          val relDataType = uc.getInput.getRowType.getFieldList.get(0).getValue
-          val logicalType = FlinkTypeFactory.toLogicalType(relDataType)
-
-          val sqlFunction =
-            BridgingSqlFunction.of(cluster, 
BuiltInFunctionDefinitions.INTERNAL_UNNEST_ROWS)
-
-          val rexCall = cluster.getRexBuilder.makeCall(
-            typeFactory.createFieldTypeFromLogicalType(
-              toRowType(UnnestRowsFunction.getUnnestedType(logicalType))),
-            sqlFunction,
-            getRel(uc.getInput).asInstanceOf[LogicalProject].getProjects
-          )
-
-          new LogicalTableFunctionScan(
-            cluster,
-            correlate.getTraitSet,
-            Collections.emptyList(),
-            rexCall,
-            null,
-            rexCall.getType,
-            null)
-      }
-    }
-
-    // convert unnest into table function scan
-    val tableFunctionScan = convert(array)
-    // create correlate with table function scan as input
-    val newCorrelate =
-      correlate.copy(correlate.getTraitSet, ImmutableList.of(outer, 
tableFunctionScan))
-    call.transformTo(newCorrelate)
-  }
-
-  private def getRel(rel: RelNode): RelNode = {
-    rel match {
-      case vertex: HepRelVertex => vertex.getCurrentRel
-      case _ => rel
-    }
-  }
-}
-
-object LogicalUnnestRule {
-  val INSTANCE = new LogicalUnnestRule(operand(classOf[LogicalCorrelate], 
any), "LogicalUnnestRule")
-}

Reply via email to