diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
new file mode 100644
index 0000000000000..4de2003afeda7
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
@@ -0,0 +1,137 @@
+/*
+ * 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.calcite.rel.logical;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.schema.Table;
+
+import java.util.List;
+
+/**
+ * A LogicalTableScan
reads all the rows from a {@link RelOptTable}.
+ *
+ *
This class is copied from Calcite because the {@link #explainTerms} should consider hints.
+ *
+ *
If the table is a net.sf.saffron.ext.JdbcTable
, then this is literally possible.
+ * But for other kinds of tables, there may be many ways to read the data from the table. For some
+ * kinds of table, it may not even be possible to read all of the rows unless some narrowing
+ * constraint is applied.
+ *
+ *
In the example of the net.sf.saffron.ext.ReflectSchema
schema,
+ *
+ *
+ *
+ * select from fields
+ *
+ *
+ *
+ * cannot be implemented, but
+ *
+ *
+ *
+ * select from fields as f
+ * where f.getClass().getName().equals("java.lang.String")
+ *
+ *
+ *
+ * can. It is the optimizer's responsibility to find these ways, by applying transformation
+ * rules.
+ */
+public final class LogicalTableScan extends TableScan {
+ // ~ Constructors -----------------------------------------------------------
+
+ /**
+ * Creates a LogicalTableScan.
+ *
+ *
Use {@link #create} unless you know what you're doing.
+ */
+ public LogicalTableScan(
+ RelOptCluster cluster, RelTraitSet traitSet, List hints, RelOptTable table) {
+ super(cluster, traitSet, hints, table);
+ }
+
+ @Deprecated // to be removed before 2.0
+ public LogicalTableScan(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) {
+ this(cluster, traitSet, ImmutableList.of(), table);
+ }
+
+ @Deprecated // to be removed before 2.0
+ public LogicalTableScan(RelOptCluster cluster, RelOptTable table) {
+ this(cluster, cluster.traitSetOf(Convention.NONE), ImmutableList.of(), table);
+ }
+
+ /** Creates a LogicalTableScan by parsing serialized output. */
+ public LogicalTableScan(RelInput input) {
+ super(input);
+ }
+
+ @Override
+ public RelNode copy(RelTraitSet traitSet, List inputs) {
+ assert traitSet.containsIfApplicable(Convention.NONE);
+ assert inputs.isEmpty();
+ return this;
+ }
+
+ // BEGIN FLINK MODIFICATION
+ // {@link #explainTerms} method should consider hints due to CALCITE-4581.
+ // This file should be remove once CALCITE-4581 is fixed.
+ @Override
+ public RelWriter explainTerms(RelWriter pw) {
+ return super.explainTerms(pw).itemIf("hints", getHints(), !getHints().isEmpty());
+ }
+ // END FLINK MODIFICATION
+
+ /**
+ * Creates a LogicalTableScan.
+ *
+ * @param cluster Cluster
+ * @param relOptTable Table
+ * @param hints The hints
+ */
+ public static LogicalTableScan create(
+ RelOptCluster cluster, final RelOptTable relOptTable, List hints) {
+ final Table table = relOptTable.unwrap(Table.class);
+ final RelTraitSet traitSet =
+ cluster.traitSetOf(Convention.NONE)
+ .replaceIfs(
+ RelCollationTraitDef.INSTANCE,
+ () -> {
+ if (table != null) {
+ return table.getStatistic().getCollations();
+ }
+ return ImmutableList.of();
+ });
+ return new LogicalTableScan(cluster, traitSet, hints, relOptTable);
+ }
+
+ @Override
+ public RelNode withHints(List hintList) {
+ return new LogicalTableScan(getCluster(), traitSet, hintList, table);
+ }
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java
index acd398f08260b..59703498cbd8a 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java
@@ -103,8 +103,9 @@ public static RelNode convertCollectToRel(
return convertSinkToRel(
relBuilder,
input,
+ Collections.emptyMap(), // dynamicOptions
collectModifyOperation.getTableIdentifier(),
- Collections.emptyMap(),
+ Collections.emptyMap(), // staticPartitions
false,
tableSink,
catalogTable);
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java
index 04fef0671f5ff..8bd61ba26cfc0 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterInCalcIntoTableSourceScanRule.java
@@ -109,7 +109,8 @@ private void pushFilterIntoScan(
TableSourceTable tableSourceTable = pushdownResultWithScan._2;
FlinkLogicalTableSourceScan newScan =
- FlinkLogicalTableSourceScan.create(scan.getCluster(), tableSourceTable);
+ FlinkLogicalTableSourceScan.create(
+ scan.getCluster(), scan.getHints(), tableSourceTable);
// build new calc program
RexProgramBuilder programBuilder =
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java
index 1aa01ccb868ea..f277ecc59eb12 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.java
@@ -83,7 +83,8 @@ public void onMatch(RelOptRuleCall call) {
TableSourceTable newTableSourceTable = applyLimit(limit, scan);
FlinkLogicalTableSourceScan newScan =
- FlinkLogicalTableSourceScan.create(scan.getCluster(), newTableSourceTable);
+ FlinkLogicalTableSourceScan.create(
+ scan.getCluster(), scan.getHints(), newTableSourceTable);
Sort newSort = sort.copy(sort.getTraitSet(), Collections.singletonList(newScan));
call.transformTo(newSort);
}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
index 1c4382ec09e84..ec2c44989c9e2 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
@@ -134,7 +134,8 @@ protected FlinkLogicalTableSourceScan getNewScan(
newType,
new String[] {digest},
new SourceAbilitySpec[] {abilitySpec});
- return FlinkLogicalTableSourceScan.create(scan.getCluster(), newTableSourceTable);
+ return FlinkLogicalTableSourceScan.create(
+ scan.getCluster(), scan.getHints(), newTableSourceTable);
}
protected boolean supportsWatermarkPushDown(FlinkLogicalTableSourceScan scan) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
index e750c6a4dcbfb..236120216eb01 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
@@ -218,10 +218,12 @@ object FlinkLogicalRelFactories {
case s: LogicalTableScan if FlinkLogicalLegacyTableSourceScan.isTableSourceScan(s) =>
FlinkLogicalLegacyTableSourceScan.create(
cluster,
+ hints,
s.getTable.asInstanceOf[FlinkPreparingTableBase])
case s: LogicalTableScan if FlinkLogicalDataStreamTableScan.isDataStreamTableScan(s) =>
FlinkLogicalDataStreamTableScan.create(
cluster,
+ hints,
s.getTable.asInstanceOf[FlinkPreparingTableBase])
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala
index d630e8569f30c..f60b4337231e1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.logical
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
import org.apache.flink.table.planner.plan.schema.DataStreamTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import com.google.common.collect.ImmutableList
import org.apache.calcite.plan._
@@ -28,11 +29,11 @@ import org.apache.calcite.rel.core.TableScan
import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.logical.LogicalTableScan
import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode}
+import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode, RelWriter}
import java.util
-import java.util.Collections
import java.util.function.Supplier
+import scala.collection.JavaConverters._
/**
* Sub-class of [[TableScan]] that is a relational operator
@@ -41,12 +42,13 @@ import java.util.function.Supplier
class FlinkLogicalDataStreamTableScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
table: RelOptTable)
- extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), table)
+ extends TableScan(cluster, traitSet, hints, table)
with FlinkLogicalRel {
override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
- new FlinkLogicalDataStreamTableScan(cluster, traitSet, table)
+ new FlinkLogicalDataStreamTableScan(cluster, traitSet, getHints, table)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
@@ -55,6 +57,12 @@ class FlinkLogicalDataStreamTableScan(
planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)
}
+ override def explainTerms(pw: RelWriter): RelWriter = {
+ super.explainTerms(pw)
+ .item("fields", getRowType.getFieldNames.asScala.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !(getHints.isEmpty));
+ }
+
}
class FlinkLogicalDataStreamTableScanConverter
@@ -72,7 +80,7 @@ class FlinkLogicalDataStreamTableScanConverter
def convert(rel: RelNode): RelNode = {
val scan = rel.asInstanceOf[TableScan]
- FlinkLogicalDataStreamTableScan.create(rel.getCluster, scan.getTable)
+ FlinkLogicalDataStreamTableScan.create(rel.getCluster, scan.getHints, scan.getTable)
}
}
@@ -84,7 +92,10 @@ object FlinkLogicalDataStreamTableScan {
dataStreamTable != null
}
- def create(cluster: RelOptCluster, relOptTable: RelOptTable): FlinkLogicalDataStreamTableScan = {
+ def create(
+ cluster: RelOptCluster,
+ hints: util.List[RelHint],
+ relOptTable: RelOptTable): FlinkLogicalDataStreamTableScan = {
val dataStreamTable = relOptTable.unwrap(classOf[DataStreamTable[_]])
val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).replaceIfs(
RelCollationTraitDef.INSTANCE, new Supplier[util.List[RelCollation]]() {
@@ -96,6 +107,6 @@ object FlinkLogicalDataStreamTableScan {
}
}
}).simplify()
- new FlinkLogicalDataStreamTableScan(cluster, traitSet, dataStreamTable)
+ new FlinkLogicalDataStreamTableScan(cluster, traitSet, hints, dataStreamTable)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalLegacyTableSourceScan.scala
index 47581d7ad0a59..98331a0472dc8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalLegacyTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalLegacyTableSourceScan.scala
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.logical
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalLegacyTableSourceScan.isTableSourceScan
import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, LegacyTableSourceTable}
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import org.apache.flink.table.sources._
import com.google.common.collect.ImmutableList
@@ -34,7 +35,6 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode, RelWriter}
import java.util
-import java.util.Collections
import java.util.function.Supplier
/**
@@ -44,8 +44,9 @@ import java.util.function.Supplier
class FlinkLogicalLegacyTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
relOptTable: LegacyTableSourceTable[_])
- extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), relOptTable)
+ extends TableScan(cluster, traitSet, hints, relOptTable)
with FlinkLogicalRel {
lazy val tableSource: TableSource[_] = tableSourceTable.tableSource
@@ -55,11 +56,11 @@ class FlinkLogicalLegacyTableSourceScan(
def copy(
traitSet: RelTraitSet,
tableSourceTable: LegacyTableSourceTable[_]): FlinkLogicalLegacyTableSourceScan = {
- new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+ new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, relOptTable)
+ new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, getHints, relOptTable)
}
override def deriveRowType(): RelDataType = {
@@ -77,6 +78,7 @@ class FlinkLogicalLegacyTableSourceScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw)
.item("fields", tableSource.getTableSchema.getFieldNames.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !getHints.isEmpty)
}
}
@@ -96,7 +98,7 @@ class FlinkLogicalLegacyTableSourceScanConverter
def convert(rel: RelNode): RelNode = {
val scan = rel.asInstanceOf[TableScan]
val table = scan.getTable.asInstanceOf[FlinkPreparingTableBase]
- FlinkLogicalLegacyTableSourceScan.create(rel.getCluster, table)
+ FlinkLogicalLegacyTableSourceScan.create(rel.getCluster, scan.getHints, table)
}
}
@@ -108,8 +110,10 @@ object FlinkLogicalLegacyTableSourceScan {
tableSourceTable != null
}
- def create(cluster: RelOptCluster, relOptTable: FlinkPreparingTableBase)
- : FlinkLogicalLegacyTableSourceScan = {
+ def create(
+ cluster: RelOptCluster,
+ hints: util.List[RelHint],
+ relOptTable: FlinkPreparingTableBase): FlinkLogicalLegacyTableSourceScan = {
val table = relOptTable.unwrap(classOf[LegacyTableSourceTable[_]])
val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).replaceIfs(
RelCollationTraitDef.INSTANCE, new Supplier[util.List[RelCollation]]() {
@@ -121,6 +125,6 @@ object FlinkLogicalLegacyTableSourceScan {
}
}
}).simplify()
- new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, table)
+ new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, hints, table)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
index 697e4575989a1..1119ed0e8a69e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
@@ -22,6 +22,7 @@ import org.apache.flink.table.connector.source.DynamicTableSource
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan.isTableSourceScan
import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import com.google.common.collect.ImmutableList
import org.apache.calcite.plan._
@@ -34,7 +35,6 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode, RelWriter}
import java.util
-import java.util.Collections
import java.util.function.Supplier
import scala.collection.JavaConversions._
@@ -46,8 +46,9 @@ import scala.collection.JavaConversions._
class FlinkLogicalTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
relOptTable: TableSourceTable)
- extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), relOptTable)
+ extends TableScan(cluster, traitSet, hints, relOptTable)
with FlinkLogicalRel {
lazy val tableSource: DynamicTableSource = relOptTable.tableSource
@@ -55,11 +56,11 @@ class FlinkLogicalTableSourceScan(
def copy(
traitSet: RelTraitSet,
tableSourceTable: TableSourceTable): FlinkLogicalTableSourceScan = {
- new FlinkLogicalTableSourceScan(cluster, traitSet, tableSourceTable)
+ new FlinkLogicalTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new FlinkLogicalTableSourceScan(cluster, traitSet, relOptTable)
+ new FlinkLogicalTableSourceScan(cluster, traitSet, getHints, relOptTable)
}
override def deriveRowType(): RelDataType = {
@@ -77,6 +78,7 @@ class FlinkLogicalTableSourceScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw)
.item("fields", getRowType.getFieldNames.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !getHints.isEmpty)
}
}
@@ -96,7 +98,7 @@ class FlinkLogicalTableSourceScanConverter
def convert(rel: RelNode): RelNode = {
val scan = rel.asInstanceOf[TableScan]
val table = scan.getTable.unwrap(classOf[TableSourceTable])
- FlinkLogicalTableSourceScan.create(rel.getCluster, table)
+ FlinkLogicalTableSourceScan.create(rel.getCluster, scan.getHints, table)
}
}
@@ -108,7 +110,10 @@ object FlinkLogicalTableSourceScan {
tableSourceTable != null
}
- def create(cluster: RelOptCluster, table: TableSourceTable): FlinkLogicalTableSourceScan = {
+ def create(
+ cluster: RelOptCluster,
+ hints: util.List[RelHint],
+ table: TableSourceTable): FlinkLogicalTableSourceScan = {
val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).replaceIfs(
RelCollationTraitDef.INSTANCE, new Supplier[util.List[RelCollation]]() {
def get: util.List[RelCollation] = {
@@ -119,6 +124,6 @@ object FlinkLogicalTableSourceScan {
}
}
}).simplify()
- new FlinkLogicalTableSourceScan(cluster, traitSet, table)
+ new FlinkLogicalTableSourceScan(cluster, traitSet, hints, table)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala
index 71e1e063c6e3c..12282936c2f0b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala
@@ -22,13 +22,17 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecBoundedStreamScan
import org.apache.flink.table.planner.plan.schema.DataStreamTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import org.apache.calcite.plan._
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
+import java.util
+
import scala.collection.JavaConverters._
/**
@@ -39,9 +43,10 @@ import scala.collection.JavaConverters._
class BatchPhysicalBoundedStreamScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
table: RelOptTable,
outputRowType: RelDataType)
- extends TableScan(cluster, traitSet, table)
+ extends TableScan(cluster, traitSet, hints, table)
with BatchPhysicalRel {
val boundedStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]])
@@ -49,7 +54,7 @@ class BatchPhysicalBoundedStreamScan(
override def deriveRowType(): RelDataType = outputRowType
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new BatchPhysicalBoundedStreamScan(cluster, traitSet, getTable, getRowType)
+ new BatchPhysicalBoundedStreamScan(cluster, traitSet, getHints, getTable, getRowType)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
@@ -61,6 +66,7 @@ class BatchPhysicalBoundedStreamScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw)
.item("fields", getRowType.getFieldNames.asScala.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !(getHints.isEmpty));
}
override def translateToExecNode(): ExecNode[_] = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala
index 85d2f22845450..8778505f8a1ab 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala
@@ -27,6 +27,7 @@ import org.apache.flink.table.sources.StreamTableSource
import org.apache.calcite.plan._
import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
import java.util
@@ -38,12 +39,13 @@ import java.util
class BatchPhysicalLegacyTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
tableSourceTable: LegacyTableSourceTable[_])
- extends CommonPhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+ extends CommonPhysicalLegacyTableSourceScan(cluster, traitSet, hints, tableSourceTable)
with BatchPhysicalRel {
override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
- new BatchPhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+ new BatchPhysicalLegacyTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala
index 9509edc33ef67..3021002c83449 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala
@@ -28,6 +28,7 @@ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil
import org.apache.calcite.plan._
import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
import java.util
@@ -39,12 +40,13 @@ import java.util
class BatchPhysicalTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
tableSourceTable: TableSourceTable)
- extends CommonPhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
+ extends CommonPhysicalTableSourceScan(cluster, traitSet, hints, tableSourceTable)
with BatchPhysicalRel {
override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
- new BatchPhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
+ new BatchPhysicalTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLegacyTableSourceScan.scala
index 47f9b6bde22ab..ad38ae751de99 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLegacyTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalLegacyTableSourceScan.scala
@@ -20,12 +20,16 @@ package org.apache.flink.table.planner.plan.nodes.physical.common
import org.apache.flink.api.dag.Transformation
import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import org.apache.flink.table.sources.TableSource
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.RelWriter
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.hint.RelHint
+
+import java.util
import scala.collection.JavaConverters._
@@ -35,8 +39,9 @@ import scala.collection.JavaConverters._
abstract class CommonPhysicalLegacyTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
relOptTable: LegacyTableSourceTable[_])
- extends TableScan(cluster, traitSet, relOptTable) {
+ extends TableScan(cluster, traitSet, hints, relOptTable) {
// cache table source transformation.
protected var sourceTransform: Transformation[_] = _
@@ -55,5 +60,6 @@ abstract class CommonPhysicalLegacyTableSourceScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw)
.item("fields", getRowType.getFieldNames.asScala.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !getHints.isEmpty);
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalTableSourceScan.scala
index 2dce72e3f865a..cd8704d6b7b56 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/common/CommonPhysicalTableSourceScan.scala
@@ -20,11 +20,15 @@ package org.apache.flink.table.planner.plan.nodes.physical.common
import org.apache.flink.table.connector.source.ScanTableSource
import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.RelWriter
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.hint.RelHint
+
+import java.util
import scala.collection.JavaConverters._
@@ -34,8 +38,9 @@ import scala.collection.JavaConverters._
abstract class CommonPhysicalTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
relOptTable: TableSourceTable)
- extends TableScan(cluster, traitSet, relOptTable) {
+ extends TableScan(cluster, traitSet, hints, relOptTable) {
protected val tableSourceTable: TableSourceTable = relOptTable.unwrap(classOf[TableSourceTable])
@@ -50,5 +55,6 @@ abstract class CommonPhysicalTableSourceScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw).item("fields", getRowType.getFieldNames.asScala.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !getHints.isEmpty)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala
index b7b45a459cc56..c80cc0d40fbbb 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala
@@ -22,13 +22,17 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDataStreamScan
import org.apache.flink.table.planner.plan.schema.DataStreamTable
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil
import org.apache.calcite.plan._
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
+import java.util
+
import scala.collection.JavaConverters._
/**
@@ -39,9 +43,10 @@ import scala.collection.JavaConverters._
class StreamPhysicalDataStreamScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
table: RelOptTable,
outputRowType: RelDataType)
- extends TableScan(cluster, traitSet, table)
+ extends TableScan(cluster, traitSet, hints, table)
with StreamPhysicalRel {
val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]])
@@ -51,7 +56,7 @@ class StreamPhysicalDataStreamScan(
override def deriveRowType(): RelDataType = outputRowType
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new StreamPhysicalDataStreamScan(cluster, traitSet, getTable, getRowType)
+ new StreamPhysicalDataStreamScan(cluster, traitSet, getHints, getTable, getRowType)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
@@ -63,6 +68,7 @@ class StreamPhysicalDataStreamScan(
override def explainTerms(pw: RelWriter): RelWriter = {
super.explainTerms(pw)
.item("fields", getRowType.getFieldNames.asScala.mkString(", "))
+ .itemIf("hints", RelExplainUtil.hintsToString(getHints), !getHints.isEmpty)
}
override def translateToExecNode(): ExecNode[_] = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala
index 2642aa831bc44..39ae30021508e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala
@@ -27,22 +27,26 @@ import org.apache.flink.table.sources.StreamTableSource
import org.apache.calcite.plan._
import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
+import java.util
+
/**
* Stream physical RelNode to read data from an external source defined by a [[StreamTableSource]].
*/
class StreamPhysicalLegacyTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
tableSourceTable: LegacyTableSourceTable[_])
- extends CommonPhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+ extends CommonPhysicalLegacyTableSourceScan(cluster, traitSet, hints, tableSourceTable)
with StreamPhysicalRel {
override def requireWatermark: Boolean = false
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new StreamPhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+ new StreamPhysicalLegacyTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala
index d5d49167b773a..992547576f66a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala
@@ -28,6 +28,7 @@ import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil
import org.apache.calcite.plan._
import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.metadata.RelMetadataQuery
import java.util
@@ -39,14 +40,15 @@ import java.util
class StreamPhysicalTableSourceScan(
cluster: RelOptCluster,
traitSet: RelTraitSet,
+ hints: util.List[RelHint],
tableSourceTable: TableSourceTable)
- extends CommonPhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
+ extends CommonPhysicalTableSourceScan(cluster, traitSet, hints, tableSourceTable)
with StreamPhysicalRel {
override def requireWatermark: Boolean = false
override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
- new StreamPhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
+ new StreamPhysicalTableSourceScan(cluster, traitSet, getHints, tableSourceTable)
}
override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalBoundedStreamScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalBoundedStreamScanRule.scala
index 9113f5238d3fe..b4389782f30d4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalBoundedStreamScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalBoundedStreamScanRule.scala
@@ -49,7 +49,12 @@ class BatchPhysicalBoundedStreamScanRule
def convert(rel: RelNode): RelNode = {
val scan = rel.asInstanceOf[FlinkLogicalDataStreamTableScan]
val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
- new BatchPhysicalBoundedStreamScan(rel.getCluster, newTrait, scan.getTable, rel.getRowType)
+ new BatchPhysicalBoundedStreamScan(
+ rel.getCluster,
+ newTrait,
+ scan.getHints,
+ scan.getTable,
+ rel.getRowType)
}
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalLegacyTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalLegacyTableSourceScanRule.scala
index 65b4a4110a696..2cb6ce3c73776 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalLegacyTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalLegacyTableSourceScanRule.scala
@@ -60,6 +60,7 @@ class BatchPhysicalLegacyTableSourceScanRule
new BatchPhysicalLegacyTableSourceScan(
rel.getCluster,
newTrait,
+ scan.getHints,
scan.getTable.asInstanceOf[LegacyTableSourceTable[_]]
)
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalTableSourceScanRule.scala
index 2ce9a1da2458a..037251123a62a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalTableSourceScanRule.scala
@@ -61,6 +61,7 @@ class BatchPhysicalTableSourceScanRule
new BatchPhysicalTableSourceScan(
rel.getCluster,
newTrait,
+ scan.getHints,
scan.getTable.asInstanceOf[TableSourceTable]
)
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalDataStreamScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalDataStreamScanRule.scala
index 2621fda5fa5f1..4d7d1ffceffc5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalDataStreamScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalDataStreamScanRule.scala
@@ -50,6 +50,7 @@ class StreamPhysicalDataStreamScanRule
new StreamPhysicalDataStreamScan(
rel.getCluster,
traitSet,
+ scan.getHints,
scan.getTable,
rel.getRowType
)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalLegacyTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalLegacyTableSourceScanRule.scala
index 580f8b7b1f6a0..3173f703971fc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalLegacyTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalLegacyTableSourceScanRule.scala
@@ -61,6 +61,7 @@ class StreamPhysicalLegacyTableSourceScanRule
new StreamPhysicalLegacyTableSourceScan(
rel.getCluster,
traitSet,
+ scan.getHints,
scan.getTable.asInstanceOf[LegacyTableSourceTable[_]]
)
}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalTableSourceScanRule.scala
index d2c81529973fa..56a8ec7d973d4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalTableSourceScanRule.scala
@@ -69,6 +69,7 @@ class StreamPhysicalTableSourceScanRule
val newScan = new StreamPhysicalTableSourceScan(
rel.getCluster,
traitSet,
+ scan.getHints,
table)
if (isUpsertSource(table.catalogTable, table.tableSource) ||
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala
index fad78bf8936f9..4d65de021731f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala
@@ -25,10 +25,11 @@ import org.apache.flink.table.planner.functions.aggfunctions.DeclarativeAggregat
import org.apache.flink.table.planner.plan.nodes.ExpressionFormat
import org.apache.flink.table.planner.plan.nodes.ExpressionFormat.ExpressionFormat
-import com.google.common.collect.ImmutableMap
+import com.google.common.collect.{ImmutableList, ImmutableMap}
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.Window.Group
import org.apache.calcite.rel.core.{AggregateCall, Window}
+import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.{RelCollation, RelWriter}
import org.apache.calcite.rex._
import org.apache.calcite.sql.SqlKind
@@ -858,4 +859,28 @@ object RelExplainUtil {
case (k, v) => s"$k = (${v.mkString(", ")})"
}.mkString(", ")
+ /**
+ * Converts [[RelHint]]s to String.
+ */
+ def hintsToString(hints: ImmutableList[RelHint]): String = {
+ val sb = new StringBuilder
+ sb.append("[")
+ hints.foreach { hint =>
+ sb.append("[").append(hint.hintName)
+ if (!hint.inheritPath.isEmpty) {
+ sb.append(" inheritPath:").append(hint.inheritPath)
+ }
+ if (hint.listOptions.size() > 0 || hint.kvOptions.size() > 0) {
+ sb.append(" options:")
+ if (hint.listOptions.size > 0) {
+ sb.append(hint.listOptions.toString)
+ } else {
+ sb.append(hint.kvOptions.toString)
+ }
+ }
+ sb.append("]")
+ }
+ sb.append("]")
+ sb.toString
+ }
}
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java
index ee57e9f8f81a0..ea90ba6040899 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java
@@ -284,6 +284,11 @@ private static RowKind parseRowKind(String rowKindShortString) {
.booleanType()
.defaultValue(false)
.withDeprecatedKeys("Option to determine whether to discard the late event.");
+ private static final ConfigOption SOURCE_NUM_ELEMENT_TO_SKIP =
+ ConfigOptions.key("source.num-element-to-skip")
+ .intType()
+ .defaultValue(-1)
+ .withDeprecatedKeys("Option to define the number of elements to skip.");
/**
* Parse partition list from Options with the format as
@@ -319,6 +324,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
boolean nestedProjectionSupported = helper.getOptions().get(NESTED_PROJECTION_SUPPORTED);
boolean enableWatermarkPushDown = helper.getOptions().get(ENABLE_WATERMARK_PUSH_DOWN);
boolean failingSource = helper.getOptions().get(FAILING_SOURCE);
+ int numElementToSkip = helper.getOptions().get(SOURCE_NUM_ELEMENT_TO_SKIP);
Optional> filterableFields =
helper.getOptions().getOptional(FILTERABLE_FIELDS);
@@ -360,6 +366,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
null,
Collections.emptyList(),
filterableFieldsSet,
+ numElementToSkip,
Long.MAX_VALUE,
partitions,
readableMetadata,
@@ -376,6 +383,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
null,
Collections.emptyList(),
filterableFieldsSet,
+ numElementToSkip,
Long.MAX_VALUE,
partitions,
readableMetadata,
@@ -395,6 +403,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
null,
Collections.emptyList(),
filterableFieldsSet,
+ numElementToSkip,
Long.MAX_VALUE,
partitions,
readableMetadata,
@@ -495,7 +504,8 @@ public Set> optionalOptions() {
SINK_CHANGELOG_MODE_ENFORCED,
WRITABLE_METADATA,
ENABLE_WATERMARK_PUSH_DOWN,
- SINK_DROP_LATE_EVENT));
+ SINK_DROP_LATE_EVENT,
+ SOURCE_NUM_ELEMENT_TO_SKIP));
}
private static int validateAndExtractRowtimeIndex(
@@ -644,6 +654,7 @@ private static class TestValuesScanTableSource
protected List filterPredicates;
protected final Set filterableFields;
protected long limit;
+ protected int numElementToSkip;
protected List