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 00000000000000..d6113743f3e441 --- /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 + * + *
+ *                            {@link BatchPhysicalCorrelate}
+ *                                   /               \
+ * empty {@link BatchPhysicalValuesRule}}     {@link FlinkLogicalTableFunctionScan}.
+ * 
+ * + *

Add the rule to support select from a UDF directly, such as the following SQL: {@code SELECT * + * FROM LATERAL TABLE(func()) as T(c)} + * + *

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)} + */ +@Value.Enclosing +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 59eb1ff168a83f..00000000000000 --- 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 e4203d8361a027..bcbadecb7ec19a 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -54,7 +54,7 @@ under the License.