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 5f1b49f0e62 [FLINK-34599][table] Migrate 
BatchPhysicalConstantTableFunctionScanRule to Java
5f1b49f0e62 is described below

commit 5f1b49f0e620592d26335c27fb08590c42632c24
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Sun Nov 24 16:31:32 2024 +0100

    [FLINK-34599][table] Migrate BatchPhysicalConstantTableFunctionScanRule to 
Java
    
    Signed-off-by: Sergey Nuyanzin <[email protected]>
---
 ...BatchPhysicalConstantTableFunctionScanRule.java | 122 +++++++++++++++++++++
 ...atchPhysicalConstantTableFunctionScanRule.scala |  82 --------------
 tools/maven/suppressions.xml                       |   2 +-
 3 files changed, 123 insertions(+), 83 deletions(-)

diff --git 
a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.java
 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.java
new file mode 100644
index 00000000000..d6113743f3e
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.java
@@ -0,0 +1,122 @@
+/*
+ * 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.physical.batch;
+
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import 
org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalCorrelate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalValues;
+
+import 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.RelTraitSet;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rex.RexUtil;
+import org.immutables.value.Value;
+
+import scala.Option;
+
+/**
+ * Converts {@link FlinkLogicalTableFunctionScan} with constant RexCall to
+ *
+ * <pre>
+ *                            {@link BatchPhysicalCorrelate}
+ *                                   /               \
+ * empty {@link BatchPhysicalValuesRule}}     {@link 
FlinkLogicalTableFunctionScan}.
+ * </pre>
+ *
+ * <p>Add the rule to support select from a UDF directly, such as the 
following SQL: {@code SELECT *
+ * FROM LATERAL TABLE(func()) as T(c)}
+ *
+ * <p>Note: {@link BatchPhysicalCorrelateRule} is responsible for converting a 
reasonable physical
+ * plan for the normal correlate query, such as the following SQL: example1: 
{@code SELECT * FROM T,
+ * LATERAL TABLE(func()) as T(c) example2: SELECT a, c FROM T, LATERAL 
TABLE(func(a)) as T(c)}
+ */
[email protected]
+public class BatchPhysicalConstantTableFunctionScanRule
+        extends RelRule<
+                BatchPhysicalConstantTableFunctionScanRule
+                        .BatchPhysicalConstantTableFunctionScanRuleConfig> {
+
+    public static final BatchPhysicalConstantTableFunctionScanRule INSTANCE =
+            BatchPhysicalConstantTableFunctionScanRuleConfig.DEFAULT.toRule();
+
+    protected BatchPhysicalConstantTableFunctionScanRule(
+            BatchPhysicalConstantTableFunctionScanRuleConfig config) {
+        super(config);
+    }
+
+    public boolean matches(RelOptRuleCall call) {
+        FlinkLogicalTableFunctionScan scan = call.rel(0);
+        return RexUtil.isConstant(scan.getCall()) && 
scan.getInputs().isEmpty();
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+        FlinkLogicalTableFunctionScan scan = call.rel(0);
+
+        // create correlate left
+        RelOptCluster cluster = scan.getCluster();
+        RelTraitSet traitSet =
+                
call.getPlanner().emptyTraitSet().replace(FlinkConventions.BATCH_PHYSICAL());
+        BatchPhysicalValues values =
+                new BatchPhysicalValues(
+                        cluster,
+                        traitSet,
+                        ImmutableList.of(ImmutableList.of()),
+                        cluster.getTypeFactory()
+                                .createStructType(ImmutableList.of(), 
ImmutableList.of()));
+
+        BatchPhysicalCorrelate correlate =
+                new BatchPhysicalCorrelate(
+                        cluster,
+                        traitSet,
+                        values,
+                        scan,
+                        Option.empty(),
+                        scan.getRowType(),
+                        JoinRelType.INNER);
+        call.transformTo(correlate);
+    }
+
+    /** Configuration for {@link BatchPhysicalConstantTableFunctionScanRule}. 
*/
+    @Value.Immutable(singleton = false)
+    public interface BatchPhysicalConstantTableFunctionScanRuleConfig extends 
RelRule.Config {
+        
BatchPhysicalConstantTableFunctionScanRule.BatchPhysicalConstantTableFunctionScanRuleConfig
+                DEFAULT =
+                        ImmutableBatchPhysicalConstantTableFunctionScanRule
+                                
.BatchPhysicalConstantTableFunctionScanRuleConfig.builder()
+                                .build()
+                                .withOperandSupplier(
+                                        b0 ->
+                                                
b0.operand(FlinkLogicalTableFunctionScan.class)
+                                                        .anyInputs())
+                                
.withDescription("BatchPhysicalConstantTableFunctionScanRule")
+                                .as(
+                                        
BatchPhysicalConstantTableFunctionScanRule
+                                                
.BatchPhysicalConstantTableFunctionScanRuleConfig
+                                                .class);
+
+        @Override
+        default BatchPhysicalConstantTableFunctionScanRule toRule() {
+            return new BatchPhysicalConstantTableFunctionScanRule(this);
+        }
+    }
+}
diff --git 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.scala
 
b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.scala
deleted file mode 100644
index 59eb1ff168a..00000000000
--- 
a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalConstantTableFunctionScanRule.scala
+++ /dev/null
@@ -1,82 +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.physical.batch
-
-import org.apache.flink.table.planner.plan.nodes.FlinkConventions
-import 
org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan
-import 
org.apache.flink.table.planner.plan.nodes.physical.batch.{BatchPhysicalCorrelate,
 BatchPhysicalValues}
-
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
-import org.apache.calcite.plan.RelOptRule._
-import org.apache.calcite.rel.core.JoinRelType
-import org.apache.calcite.rex.{RexLiteral, RexUtil}
-
-/**
- * Converts [[FlinkLogicalTableFunctionScan]] with constant RexCall to
- * {{{
- *                         [[BatchPhysicalCorrelate]]
- *                            /             \
- * empty [[BatchPhysicalValuesRule]]  [[FlinkLogicalTableFunctionScan]]
- * }}}
- *
- * Add the rule to support select from a UDF directly, such as the following 
SQL: SELECT * FROM
- * LATERAL TABLE(func()) as T(c)
- *
- * Note: [[BatchPhysicalCorrelateRule]] is responsible for converting a 
reasonable physical plan for
- * the normal correlate query, such as the following SQL: example1: SELECT * 
FROM T, LATERAL
- * TABLE(func()) as T(c) example2: SELECT a, c FROM T, LATERAL TABLE(func(a)) 
as T(c)
- */
-class BatchPhysicalConstantTableFunctionScanRule
-  extends RelOptRule(
-    operand(classOf[FlinkLogicalTableFunctionScan], any),
-    "BatchPhysicalConstantTableFunctionScanRule") {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val scan: FlinkLogicalTableFunctionScan = call.rel(0)
-    RexUtil.isConstant(scan.getCall) && scan.getInputs.isEmpty
-  }
-
-  override def onMatch(call: RelOptRuleCall): Unit = {
-    val scan: FlinkLogicalTableFunctionScan = call.rel(0)
-
-    // create correlate left
-    val cluster = scan.getCluster
-    val traitSet = 
call.getPlanner.emptyTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
-    val values = new BatchPhysicalValues(
-      cluster,
-      traitSet,
-      ImmutableList.of(ImmutableList.of[RexLiteral]()),
-      cluster.getTypeFactory.createStructType(ImmutableList.of(), 
ImmutableList.of()))
-
-    val correlate = new BatchPhysicalCorrelate(
-      cluster,
-      traitSet,
-      values,
-      scan,
-      None,
-      scan.getRowType,
-      JoinRelType.INNER)
-    call.transformTo(correlate)
-  }
-
-}
-
-object BatchPhysicalConstantTableFunctionScanRule {
-  val INSTANCE = new BatchPhysicalConstantTableFunctionScanRule
-}
diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml
index 498355b3a01..c90930331e9 100644
--- a/tools/maven/suppressions.xml
+++ b/tools/maven/suppressions.xml
@@ -56,7 +56,7 @@ under the License.
 
            <!-- Have to use guava directly -->
            <suppress
-                       
files="OverConvertRule.java|InConverter.java|SymbolUtil.java|RexNodeJsonDeserializer.java|RexNodeJsonSerializer.java|RexNodeJsonSerdeTest.java|FlinkAggregateProjectMergeRule.java|StreamPhysicalConstantTableFunctionScanRule.java"
+                       
files="OverConvertRule.java|InConverter.java|SymbolUtil.java|RexNodeJsonDeserializer.java|RexNodeJsonSerializer.java|RexNodeJsonSerdeTest.java|FlinkAggregateProjectMergeRule.java|BatchPhysicalConstantTableFunctionScanRule.java|StreamPhysicalConstantTableFunctionScanRule.java"
                        checks="IllegalImport"/>
                <!-- Classes copied from AWS -->
                <suppress

Reply via email to