You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2020/05/09 02:01:51 UTC

[flink] 02/03: [FLINK-16989][table-planner-blink] Rename current TableSourceScan into LegacyTableSourceScan

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

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

commit f2d102902b4dcabf958f32d32117301991bd5d0d
Author: Jark Wu <ja...@apache.org>
AuthorDate: Sat May 2 19:53:56 2020 +0800

    [FLINK-16989][table-planner-blink] Rename current TableSourceScan into LegacyTableSourceScan
    
    This closes #11985
---
 .../planner/plan/FlinkCalciteCatalogReader.java    |   4 +-
 .../planner/plan/QueryOperationConverter.java      |   4 +-
 .../planner/calcite/FlinkLogicalRelFactories.scala |   6 +-
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |   2 +-
 .../plan/metadata/FlinkRelMdUniqueKeys.scala       |   2 +-
 ...ala => FlinkLogicalLegacyTableSourceScan.scala} |  34 +-
 ...n.scala => PhysicalLegacyTableSourceScan.scala} |  10 +-
 ....scala => BatchExecLegacyTableSourceScan.scala} |  16 +-
 ...scala => StreamExecLegacyTableSourceScan.scala} |  16 +-
 .../FlinkChangelogModeInferenceProgram.scala       |   6 +-
 .../table/planner/plan/reuse/SubplanReuser.scala   |   6 +-
 .../planner/plan/rules/FlinkBatchRuleSets.scala    |  14 +-
 .../planner/plan/rules/FlinkStreamRuleSets.scala   |  12 +-
 ... PushFilterIntoLegacyTableSourceScanRule.scala} |  22 +-
 ...> PushLimitIntoLegacyTableSourceScanRule.scala} |  29 +-
 ...shPartitionIntoLegacyTableSourceScanRule.scala} |  16 +-
 ...PushProjectIntoLegacyTableSourceScanRule.scala} |  16 +-
 ...la => BatchExecLegacyTableSourceScanRule.scala} |  28 +-
 .../physical/common/CommonLookupJoinRule.scala     |  10 +-
 .../stream/MiniBatchIntervalInferRule.scala        |   4 +-
 ...a => StreamExecLegacyTableSourceScanRule.scala} |  28 +-
 .../planner/plan/schema/CatalogSourceTable.scala   |  12 +-
 ...rceTable.scala => LegacyTableSourceTable.scala} |  10 +-
 .../explain/testSqlUpdateAndToDataStream.out       |   2 +-
 .../explain/testStreamTableEnvironmentExplain.out  |   2 +-
 .../apache/flink/table/api/batch/ExplainTest.xml   |   4 +-
 .../apache/flink/table/api/stream/ExplainTest.xml  |   4 +-
 .../flink/table/executor/BatchExecutorTest.xml     |   4 +-
 .../table/planner/plan/batch/sql/CalcTest.xml      |  38 +-
 .../planner/plan/batch/sql/DagOptimizationTest.xml |  60 ++--
 .../planner/plan/batch/sql/DeadlockBreakupTest.xml |  16 +-
 .../table/planner/plan/batch/sql/LimitTest.xml     |  22 +-
 .../plan/batch/sql/PartitionableSinkTest.xml       |   8 +-
 .../table/planner/plan/batch/sql/RankTest.xml      |  14 +-
 .../planner/plan/batch/sql/RemoveCollationTest.xml |  78 ++--
 .../planner/plan/batch/sql/RemoveShuffleTest.xml   | 156 ++++----
 .../planner/plan/batch/sql/SetOperatorsTest.xml    |  28 +-
 .../table/planner/plan/batch/sql/SortLimitTest.xml |  12 +-
 .../table/planner/plan/batch/sql/SortTest.xml      |  10 +-
 .../planner/plan/batch/sql/SubplanReuseTest.xml    |  86 ++---
 .../planner/plan/batch/sql/TableSourceTest.xml     |  30 +-
 .../table/planner/plan/batch/sql/UnionTest.xml     |  10 +-
 .../table/planner/plan/batch/sql/UnnestTest.xml    |  14 +-
 .../batch/sql/agg/AggregateReduceGroupingTest.xml  | 106 +++---
 .../plan/batch/sql/agg/DistinctAggregateTest.xml   |  48 +--
 .../plan/batch/sql/agg/GroupingSetsTest.xml        |  72 ++--
 .../plan/batch/sql/agg/HashAggregateTest.xml       |  84 ++---
 .../plan/batch/sql/agg/OverAggregateTest.xml       |  46 +--
 .../plan/batch/sql/agg/SortAggregateTest.xml       | 102 +++---
 .../plan/batch/sql/agg/WindowAggregateTest.xml     | 126 +++----
 .../plan/batch/sql/join/BroadcastHashJoinTest.xml  |  54 +--
 .../sql/join/BroadcastHashSemiAntiJoinTest.xml     | 294 +++++++--------
 .../plan/batch/sql/join/JoinReorderTest.xml        | 130 +++----
 .../planner/plan/batch/sql/join/LookupJoinTest.xml |   2 +-
 .../plan/batch/sql/join/NestedLoopJoinTest.xml     | 110 +++---
 .../batch/sql/join/NestedLoopSemiAntiJoinTest.xml  | 394 ++++++++++-----------
 .../plan/batch/sql/join/SemiAntiJoinTest.xml       | 394 ++++++++++-----------
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   |  68 ++--
 .../sql/join/ShuffledHashSemiAntiJoinTest.xml      | 306 ++++++++--------
 .../plan/batch/sql/join/SingleRowJoinTest.xml      |  26 +-
 .../plan/batch/sql/join/SortMergeJoinTest.xml      |  70 ++--
 .../batch/sql/join/SortMergeSemiAntiJoinTest.xml   | 306 ++++++++--------
 .../planner/plan/batch/table/AggregateTest.xml     |   8 +-
 .../table/planner/plan/batch/table/CalcTest.xml    |  26 +-
 .../plan/batch/table/ColumnFunctionsTest.xml       |   2 +-
 .../planner/plan/batch/table/CorrelateTest.xml     |  14 +-
 .../planner/plan/batch/table/GroupWindowTest.xml   |  12 +-
 .../table/planner/plan/batch/table/JoinTest.xml    |  52 +--
 .../planner/plan/batch/table/PythonCalcTest.xml    |   2 +-
 .../planner/plan/batch/table/SetOperatorsTest.xml  |  24 +-
 .../stringexpr/CorrelateStringExpressionTest.xml   |  16 +-
 .../batch/table/stringexpr/SetOperatorsTest.xml    |   4 +-
 .../table/planner/plan/hint/OptionsHintTest.xml    |  40 +--
 .../logical/AggregateReduceGroupingRuleTest.xml    | 106 +++---
 .../CalcPythonCorrelateTransposeRuleTest.xml       |   2 +-
 .../rules/logical/CalcRankTransposeRuleTest.xml    |   6 +-
 .../logical/DecomposeGroupingSetsRuleTest.xml      |  16 +-
 .../rules/logical/ExpressionReductionRulesTest.xml |   6 +-
 ...nkAggregateExpandDistinctAggregatesRuleTest.xml |  52 +--
 .../rules/logical/FlinkAggregateRemoveRuleTest.xml |  46 +--
 .../plan/rules/logical/FlinkCalcMergeRuleTest.xml  |   6 +-
 .../FlinkLogicalRankRuleForConstantRangeTest.xml   |  30 +-
 .../FlinkLogicalRankRuleForRangeEndTest.xml        |  26 +-
 ...ushFilterIntoLegacyTableSourceScanRuleTest.xml} |  76 ++--
 ...PartitionIntoLegacyTableSourceScanRuleTest.xml} |   0
 ...shProjectIntoLegacyTableSourceScanRuleTest.xml} |  24 --
 .../plan/rules/logical/PythonCalcSplitRuleTest.xml |  44 +--
 .../rules/logical/PythonCorrelateSplitRuleTest.xml |   8 +-
 .../plan/rules/logical/SplitAggregateRuleTest.xml  |  34 +-
 .../SplitPythonConditionFromCorrelateRuleTest.xml  |   2 +-
 .../SplitPythonConditionFromJoinRuleTest.xml       |   8 +-
 .../physical/batch/EnforceLocalHashAggRuleTest.xml |  12 +-
 .../physical/batch/EnforceLocalSortAggRuleTest.xml |  18 +-
 .../batch/RemoveRedundantLocalHashAggRuleTest.xml  |   8 +-
 .../batch/RemoveRedundantLocalRankRuleTest.xml     |   6 +-
 .../batch/RemoveRedundantLocalSortAggRuleTest.xml  |  10 +-
 .../physical/stream/ChangelogModeInferenceTest.xml |  20 +-
 .../table/planner/plan/stream/sql/CalcTest.xml     |  40 +--
 .../plan/stream/sql/DagOptimizationTest.xml        |  66 ++--
 .../plan/stream/sql/PartitionableSinkTest.xml      |   8 +-
 .../table/planner/plan/stream/sql/RankTest.xml     |   2 +-
 .../planner/plan/stream/sql/SetOperatorsTest.xml   |  28 +-
 .../planner/plan/stream/sql/SubplanReuseTest.xml   |  58 +--
 .../planner/plan/stream/sql/TableSourceTest.xml    |  44 +--
 .../table/planner/plan/stream/sql/UnionTest.xml    |  10 +-
 .../table/planner/plan/stream/sql/UnnestTest.xml   |  14 +-
 .../planner/plan/stream/sql/agg/AggregateTest.xml  |  36 +-
 .../plan/stream/sql/agg/DistinctAggregateTest.xml  | 128 +++----
 .../plan/stream/sql/agg/GroupingSetsTest.xml       |  70 ++--
 .../stream/sql/agg/IncrementalAggregateTest.xml    |  32 +-
 .../plan/stream/sql/agg/TwoStageAggregateTest.xml  |  14 +-
 .../plan/stream/sql/agg/WindowAggregateTest.xml    |   6 +-
 .../plan/stream/sql/join/JoinReorderTest.xml       | 130 +++----
 .../planner/plan/stream/sql/join/JoinTest.xml      | 138 ++++----
 .../plan/stream/sql/join/SemiAntiJoinTest.xml      | 390 ++++++++++----------
 .../planner/plan/stream/table/AggregateTest.xml    |  24 +-
 .../table/planner/plan/stream/table/CalcTest.xml   |  18 +-
 .../plan/stream/table/ColumnFunctionsTest.xml      |  26 +-
 .../planner/plan/stream/table/CorrelateTest.xml    |  24 +-
 .../stream/table/GroupWindowTableAggregateTest.xml |   6 +-
 .../planner/plan/stream/table/GroupWindowTest.xml  |   2 +-
 .../planner/plan/stream/table/PythonCalcTest.xml   |   2 +-
 .../planner/plan/stream/table/SetOperatorsTest.xml |  22 +-
 .../plan/stream/table/TableAggregateTest.xml       |  14 +-
 .../planner/plan/stream/table/TableSourceTest.xml  |  18 +-
 .../plan/stream/table/TwoStageAggregateTest.xml    |  12 +-
 .../logical/FlinkAggregateRemoveRuleTest.scala     |   4 +-
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   4 +-
 ...hFilterIntoLegacyTableSourceScanRuleTest.scala} |   6 +-
 ...rtitionIntoLegacyTableSourceScanRuleTest.scala} |   8 +-
 ...ProjectIntoLegacyTableSourceScanRuleTest.scala} |   6 +-
 131 files changed, 2899 insertions(+), 2920 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java
index 7552714..c8a18bf 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java
@@ -30,7 +30,7 @@ import org.apache.flink.table.planner.catalog.QueryOperationCatalogViewTable;
 import org.apache.flink.table.planner.catalog.SqlCatalogViewTable;
 import org.apache.flink.table.planner.plan.schema.CatalogSourceTable;
 import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase;
-import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable;
 import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
 import org.apache.flink.table.sources.LookupableTableSource;
 import org.apache.flink.table.sources.StreamTableSource;
@@ -172,7 +172,7 @@ public class FlinkCalciteCatalogReader extends CalciteCatalogReader {
 			throw new ValidationException("Only bounded StreamTableSource can be used in batch mode.");
 		}
 
-		return new TableSourceTable<>(
+		return new LegacyTableSourceTable<>(
 			relOptSchema,
 			tableIdentifier,
 			rowType,
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
index d2bea44..fbb139c 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
@@ -78,7 +78,7 @@ import org.apache.flink.table.planner.plan.logical.SlidingGroupWindow;
 import org.apache.flink.table.planner.plan.logical.TumblingGroupWindow;
 import org.apache.flink.table.planner.plan.schema.DataStreamTable;
 import org.apache.flink.table.planner.plan.schema.DataStreamTable$;
-import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable;
 import org.apache.flink.table.planner.plan.schema.TypedFlinkTableFunction;
 import org.apache.flink.table.planner.plan.stats.FlinkStatistic;
 import org.apache.flink.table.planner.sources.TableSourceUtil;
@@ -466,7 +466,7 @@ public class QueryOperationConverter extends QueryOperationDefaultVisitor<RelNod
 				tableSourceOperation.getTableSchema(),
 				scala.Option.apply(tableSource),
 				!isBatch);
-			TableSourceTable<?> tableSourceTable = new TableSourceTable<>(
+			LegacyTableSourceTable<?> tableSourceTable = new LegacyTableSourceTable<>(
 				relBuilder.getRelOptSchema(),
 				tableIdentifier,
 				rowType,
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 56ddbf0..fb42eab 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
@@ -210,7 +210,7 @@ object FlinkLogicalRelFactories {
 
   /**
     * Implementation of [[TableScanFactory]] that returns a
-    * [[FlinkLogicalTableSourceScan]] or [[FlinkLogicalDataStreamTableScan]].
+    * [[FlinkLogicalLegacyTableSourceScan]] or [[FlinkLogicalDataStreamTableScan]].
     */
   class TableScanFactoryImpl extends TableScanFactory {
     def createScan(toRelContext: ToRelContext, table: RelOptTable): RelNode = {
@@ -218,8 +218,8 @@ object FlinkLogicalRelFactories {
       val hints = toRelContext.getTableHints
       val tableScan = LogicalTableScan.create(cluster, table, hints)
       tableScan match {
-        case s: LogicalTableScan if FlinkLogicalTableSourceScan.isTableSourceScan(s) =>
-          FlinkLogicalTableSourceScan.create(
+        case s: LogicalTableScan if FlinkLogicalLegacyTableSourceScan.isTableSourceScan(s) =>
+          FlinkLogicalLegacyTableSourceScan.create(
             cluster,
             s.getTable.asInstanceOf[FlinkPreparingTableBase])
         case s: LogicalTableScan if FlinkLogicalDataStreamTableScan.isDataStreamTableScan(s) =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
index c1d1b93..a73f26a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
@@ -65,7 +65,7 @@ class FlinkRelMdColumnUniqueness private extends MetadataHandler[BuiltInMetadata
   }
 
   def areColumnsUnique(
-      rel: FlinkLogicalTableSourceScan,
+      rel: FlinkLogicalLegacyTableSourceScan,
       mq: RelMetadataQuery,
       columns: ImmutableBitSet,
       ignoreNulls: Boolean): JBoolean = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala
index 4efddb4..3dcb7dc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala
@@ -61,7 +61,7 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
   }
 
   def getUniqueKeys(
-      rel: FlinkLogicalTableSourceScan,
+      rel: FlinkLogicalLegacyTableSourceScan,
       mq: RelMetadataQuery,
       ignoreNulls: Boolean): JSet[ImmutableBitSet] = {
     getTableUniqueKeys(rel.tableSource, rel.getTable)
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/FlinkLogicalLegacyTableSourceScan.scala
similarity index 79%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalLegacyTableSourceScan.scala
index 396318a7..47581d7 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/FlinkLogicalLegacyTableSourceScan.scala
@@ -19,8 +19,8 @@
 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.FlinkLogicalTableSourceScan.isTableSourceScan
-import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, TableSourceTable}
+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.sources._
 
 import com.google.common.collect.ImmutableList
@@ -41,25 +41,25 @@ import java.util.function.Supplier
   * Sub-class of [[TableScan]] that is a relational operator
   * which returns the contents of a [[TableSource]] in Flink.
   */
-class FlinkLogicalTableSourceScan(
+class FlinkLogicalLegacyTableSourceScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
-    relOptTable: TableSourceTable[_])
+    relOptTable: LegacyTableSourceTable[_])
   extends TableScan(cluster, traitSet, Collections.emptyList[RelHint](), relOptTable)
   with FlinkLogicalRel {
 
   lazy val tableSource: TableSource[_] = tableSourceTable.tableSource
 
-  private lazy val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[_]])
+  private lazy val tableSourceTable = relOptTable.unwrap(classOf[LegacyTableSourceTable[_]])
 
   def copy(
       traitSet: RelTraitSet,
-      tableSourceTable: TableSourceTable[_]): FlinkLogicalTableSourceScan = {
-    new FlinkLogicalTableSourceScan(cluster, traitSet, tableSourceTable)
+      tableSourceTable: LegacyTableSourceTable[_]): FlinkLogicalLegacyTableSourceScan = {
+    new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
   }
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new FlinkLogicalTableSourceScan(cluster, traitSet, relOptTable)
+    new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, relOptTable)
   }
 
   override def deriveRowType(): RelDataType = {
@@ -81,12 +81,12 @@ class FlinkLogicalTableSourceScan(
 
 }
 
-class FlinkLogicalTableSourceScanConverter
+class FlinkLogicalLegacyTableSourceScanConverter
   extends ConverterRule(
     classOf[LogicalTableScan],
     Convention.NONE,
     FlinkConventions.LOGICAL,
-    "FlinkLogicalTableSourceScanConverter") {
+    "FlinkLogicalLegacyTableSourceScanConverter") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val scan: TableScan = call.rel(0)
@@ -96,21 +96,21 @@ class FlinkLogicalTableSourceScanConverter
   def convert(rel: RelNode): RelNode = {
     val scan = rel.asInstanceOf[TableScan]
     val table = scan.getTable.asInstanceOf[FlinkPreparingTableBase]
-    FlinkLogicalTableSourceScan.create(rel.getCluster, table)
+    FlinkLogicalLegacyTableSourceScan.create(rel.getCluster, table)
   }
 }
 
-object FlinkLogicalTableSourceScan {
-  val CONVERTER = new FlinkLogicalTableSourceScanConverter
+object FlinkLogicalLegacyTableSourceScan {
+  val CONVERTER = new FlinkLogicalLegacyTableSourceScanConverter
 
   def isTableSourceScan(scan: TableScan): Boolean = {
-    val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     tableSourceTable != null
   }
 
   def create(cluster: RelOptCluster, relOptTable: FlinkPreparingTableBase)
-      : FlinkLogicalTableSourceScan = {
-    val table = relOptTable.unwrap(classOf[TableSourceTable[_]])
+      : FlinkLogicalLegacyTableSourceScan = {
+    val table = relOptTable.unwrap(classOf[LegacyTableSourceTable[_]])
     val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).replaceIfs(
       RelCollationTraitDef.INSTANCE, new Supplier[util.List[RelCollation]]() {
         def get: util.List[RelCollation] = {
@@ -121,6 +121,6 @@ object FlinkLogicalTableSourceScan {
           }
         }
       }).simplify()
-    new FlinkLogicalTableSourceScan(cluster, traitSet, table)
+    new FlinkLogicalLegacyTableSourceScan(cluster, traitSet, table)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalLegacyTableSourceScan.scala
similarity index 92%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalLegacyTableSourceScan.scala
index 5c4c7ba..0ba2549 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalLegacyTableSourceScan.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.core.io.InputSplit
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 import org.apache.flink.table.sources.{InputFormatTableSource, StreamTableSource, TableSource}
 
@@ -37,17 +37,17 @@ import scala.collection.JavaConverters._
 /**
   * Base physical RelNode to read data from an external source defined by a [[TableSource]].
   */
-abstract class PhysicalTableSourceScan(
+abstract class PhysicalLegacyTableSourceScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
-    relOptTable: TableSourceTable[_])
+    relOptTable: LegacyTableSourceTable[_])
   extends TableScan(cluster, traitSet, relOptTable) {
 
   // cache table source transformation.
   protected var sourceTransform: Transformation[_] = _
 
-  protected val tableSourceTable: TableSourceTable[_] =
-    relOptTable.unwrap(classOf[TableSourceTable[_]])
+  protected val tableSourceTable: LegacyTableSourceTable[_] =
+    relOptTable.unwrap(classOf[LegacyTableSourceTable[_]])
 
   protected[flink] val tableSource: TableSource[_] = tableSourceTable.tableSource
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLegacyTableSourceScan.scala
similarity index 94%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLegacyTableSourceScan.scala
index 353124a..4c08faf 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLegacyTableSourceScan.scala
@@ -32,8 +32,8 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext
 import org.apache.flink.table.planner.delegation.BatchPlanner
 import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode}
-import org.apache.flink.table.planner.plan.nodes.physical.PhysicalTableSourceScan
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.nodes.physical.PhysicalLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.planner.plan.utils.ScanUtil
 import org.apache.flink.table.planner.sources.TableSourceUtil
 import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter
@@ -54,16 +54,16 @@ import scala.collection.JavaConversions._
   * Batch physical RelNode to read data from an external source defined by a
   * bounded [[StreamTableSource]].
   */
-class BatchExecTableSourceScan(
+class BatchExecLegacyTableSourceScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
-    tableSourceTable: TableSourceTable[_])
-  extends PhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
-  with BatchPhysicalRel
-  with BatchExecNode[RowData]{
+    tableSourceTable: LegacyTableSourceTable[_])
+  extends PhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+          with BatchPhysicalRel
+          with BatchExecNode[RowData]{
 
   override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
-    new BatchExecTableSourceScan(cluster, traitSet, tableSourceTable)
+    new BatchExecLegacyTableSourceScan(cluster, traitSet, 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/StreamExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLegacyTableSourceScan.scala
similarity index 96%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLegacyTableSourceScan.scala
index 08a21b4..1ec3b72 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLegacyTableSourceScan.scala
@@ -35,8 +35,8 @@ import org.apache.flink.table.planner.codegen.CodeGeneratorContext
 import org.apache.flink.table.planner.codegen.OperatorCodeGenerator._
 import org.apache.flink.table.planner.delegation.StreamPlanner
 import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, StreamExecNode}
-import org.apache.flink.table.planner.plan.nodes.physical.PhysicalTableSourceScan
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.nodes.physical.PhysicalLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.planner.plan.utils.ScanUtil
 import org.apache.flink.table.planner.sources.TableSourceUtil
 import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator
@@ -60,18 +60,18 @@ import scala.collection.JavaConversions._
 /**
   * Stream physical RelNode to read data from an external source defined by a [[StreamTableSource]].
   */
-class StreamExecTableSourceScan(
+class StreamExecLegacyTableSourceScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
-    tableSourceTable: TableSourceTable[_])
-  extends PhysicalTableSourceScan(cluster, traitSet, tableSourceTable)
-  with StreamPhysicalRel
-  with StreamExecNode[RowData] {
+    tableSourceTable: LegacyTableSourceTable[_])
+  extends PhysicalLegacyTableSourceScan(cluster, traitSet, tableSourceTable)
+          with StreamPhysicalRel
+          with StreamExecNode[RowData] {
 
   override def requireWatermark: Boolean = false
 
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new StreamExecTableSourceScan(cluster, traitSet, tableSourceTable)
+    new StreamExecLegacyTableSourceScan(cluster, traitSet, 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/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
index 0a9860d..7527cf3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
@@ -258,7 +258,8 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
         createNewNode(
           union, children, new ModifyKindSetTrait(providedKindSet), requiredTrait, requester)
 
-      case _: StreamExecDataStreamScan | _: StreamExecTableSourceScan | _: StreamExecValues =>
+      case _: StreamExecDataStreamScan | _: StreamExecLegacyTableSourceScan |
+           _: StreamExecValues =>
         // DataStream, TableSource and Values only support producing insert-only messages
         createNewNode(
           rel, List(), ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester)
@@ -535,7 +536,8 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
           createNewNode(union, Some(children.flatten), providedTrait)
         }
 
-      case _: StreamExecDataStreamScan | _: StreamExecTableSourceScan | _: StreamExecValues =>
+      case _: StreamExecDataStreamScan | _: StreamExecLegacyTableSourceScan |
+           _: StreamExecValues =>
         createNewNode(rel, Some(List()), UpdateKindTrait.NONE)
 
       case scan: StreamExecIntermediateTableScan =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
index cf7404a..865c235 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.planner.plan.reuse
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.planner.plan.nodes.calcite.Sink
-import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan
-import org.apache.flink.table.planner.plan.nodes.physical.PhysicalTableSourceScan
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.nodes.physical.PhysicalLegacyTableSourceScan
 import org.apache.flink.table.planner.plan.utils.{DefaultRelShuttle, FlinkRelOptUtil}
 
 import com.google.common.collect.{Maps, Sets}
@@ -152,7 +152,7 @@ object SubplanReuser {
     private def isNodeReusableDisabled(node: RelNode): Boolean = {
       node match {
         // TableSourceScan node can not be reused if reuse TableSource disabled
-        case _: FlinkLogicalTableSourceScan | _: PhysicalTableSourceScan => !tableSourceReuseEnabled
+        case _: FlinkLogicalLegacyTableSourceScan | _: PhysicalLegacyTableSourceScan => !tableSourceReuseEnabled
         // Exchange node can not be reused if its input is reusable disabled
         case e: Exchange => isNodeReusableDisabled(e.getInput)
         // TableFunctionScan and sink can not be reused
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
index ef8bb8c..32e91d3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
@@ -90,7 +90,7 @@ object FlinkBatchRuleSets {
 
   private val LIMIT_RULES: RuleSet = RuleSets.ofList(
     //push down localLimit
-    PushLimitIntoTableSourceScanRule.INSTANCE)
+    PushLimitIntoLegacyTableSourceScanRule.INSTANCE)
 
   /**
     * RuleSet to simplify predicate expressions in filters and joins
@@ -165,9 +165,9 @@ object FlinkBatchRuleSets {
     */
   val FILTER_TABLESCAN_PUSHDOWN_RULES: RuleSet = RuleSets.ofList(
     // push a filter down into the table scan
-    PushFilterIntoTableSourceScanRule.INSTANCE,
+    PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
     // push partition into the table scan
-    PushPartitionIntoTableSourceScanRule.INSTANCE
+    PushPartitionIntoLegacyTableSourceScanRule.INSTANCE
   )
 
   /**
@@ -245,8 +245,8 @@ object FlinkBatchRuleSets {
     */
   private val LOGICAL_RULES: RuleSet = RuleSets.ofList(
     // scan optimization
-    PushProjectIntoTableSourceScanRule.INSTANCE,
-    PushFilterIntoTableSourceScanRule.INSTANCE,
+    PushProjectIntoLegacyTableSourceScanRule.INSTANCE,
+    PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
 
     // reorder sort and projection
     SortProjectTransposeRule.INSTANCE,
@@ -326,7 +326,7 @@ object FlinkBatchRuleSets {
     FlinkLogicalSort.BATCH_CONVERTER,
     FlinkLogicalUnion.CONVERTER,
     FlinkLogicalValues.CONVERTER,
-    FlinkLogicalTableSourceScan.CONVERTER,
+    FlinkLogicalLegacyTableSourceScan.CONVERTER,
     FlinkLogicalTableFunctionScan.CONVERTER,
     FlinkLogicalDataStreamTableScan.CONVERTER,
     FlinkLogicalIntermediateTableScan.CONVERTER,
@@ -382,7 +382,7 @@ object FlinkBatchRuleSets {
     FlinkExpandConversionRule.BATCH_INSTANCE,
     // source
     BatchExecBoundedStreamScanRule.INSTANCE,
-    BatchExecTableSourceScanRule.INSTANCE,
+    BatchExecLegacyTableSourceScanRule.INSTANCE,
     BatchExecIntermediateTableScanRule.INSTANCE,
     BatchExecValuesRule.INSTANCE,
     // calc
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
index 91d93d2..7429178 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
@@ -162,9 +162,9 @@ object FlinkStreamRuleSets {
     */
   val FILTER_TABLESCAN_PUSHDOWN_RULES: RuleSet = RuleSets.ofList(
     // push a filter down into the table scan
-    PushFilterIntoTableSourceScanRule.INSTANCE,
+    PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
     // push partition into the table scan
-    PushPartitionIntoTableSourceScanRule.INSTANCE
+    PushPartitionIntoLegacyTableSourceScanRule.INSTANCE
   )
 
   /**
@@ -226,8 +226,8 @@ object FlinkStreamRuleSets {
     */
   private val LOGICAL_RULES: RuleSet = RuleSets.ofList(
     // scan optimization
-    PushProjectIntoTableSourceScanRule.INSTANCE,
-    PushFilterIntoTableSourceScanRule.INSTANCE,
+    PushProjectIntoLegacyTableSourceScanRule.INSTANCE,
+    PushFilterIntoLegacyTableSourceScanRule.INSTANCE,
 
     // reorder sort and projection
     SortProjectTransposeRule.INSTANCE,
@@ -299,7 +299,7 @@ object FlinkStreamRuleSets {
     FlinkLogicalSort.STREAM_CONVERTER,
     FlinkLogicalUnion.CONVERTER,
     FlinkLogicalValues.CONVERTER,
-    FlinkLogicalTableSourceScan.CONVERTER,
+    FlinkLogicalLegacyTableSourceScan.CONVERTER,
     FlinkLogicalTableFunctionScan.CONVERTER,
     FlinkLogicalDataStreamTableScan.CONVERTER,
     FlinkLogicalIntermediateTableScan.CONVERTER,
@@ -364,7 +364,7 @@ object FlinkStreamRuleSets {
     FlinkExpandConversionRule.STREAM_INSTANCE,
     // source
     StreamExecDataStreamScanRule.INSTANCE,
-    StreamExecTableSourceScanRule.INSTANCE,
+    StreamExecLegacyTableSourceScanRule.INSTANCE,
     StreamExecIntermediateTableScanRule.INSTANCE,
     StreamExecWatermarkAssignerRule.INSTANCE,
     StreamExecValuesRule.INSTANCE,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
similarity index 88%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
index f247f30..1e673d8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.api.TableException
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.planner.calcite.FlinkContext
 import org.apache.flink.table.planner.expressions.converter.ExpressionConverter
-import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, TableSourceTable}
+import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, LegacyTableSourceTable}
 import org.apache.flink.table.planner.plan.stats.FlinkStatistic
 import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RexNodeExtractor}
 import org.apache.flink.table.sources.FilterableTableSource
@@ -42,10 +42,10 @@ import scala.collection.JavaConversions._
 /**
   * Planner rule that tries to push a filter into a [[FilterableTableSource]].
   */
-class PushFilterIntoTableSourceScanRule extends RelOptRule(
+class PushFilterIntoLegacyTableSourceScanRule extends RelOptRule(
   operand(classOf[Filter],
     operand(classOf[LogicalTableScan], none)),
-  "PushFilterIntoTableSourceScanRule") {
+  "PushFilterIntoLegacyTableSourceScanRule") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val config = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
@@ -60,8 +60,8 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
     }
 
     val scan: LogicalTableScan = call.rel(1)
-    scan.getTable.unwrap(classOf[TableSourceTable[_]]) match {
-      case table: TableSourceTable[_] =>
+    scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]]) match {
+      case table: LegacyTableSourceTable[_] =>
         table.tableSource match {
           case source: FilterableTableSource[_] => !source.isFilterPushedDown
           case _ => false
@@ -73,7 +73,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
   override def onMatch(call: RelOptRuleCall): Unit = {
     val filter: Filter = call.rel(0)
     val scan: LogicalTableScan = call.rel(1)
-    val table: TableSourceTable[_] = scan.getTable.asInstanceOf[TableSourceTable[_]]
+    val table: LegacyTableSourceTable[_] = scan.getTable.asInstanceOf[LegacyTableSourceTable[_]]
     pushFilterIntoScan(call, filter, scan, table)
   }
 
@@ -107,8 +107,8 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
 
     val newRelOptTable: FlinkPreparingTableBase =
       applyPredicate(remainingPredicates, relOptTable, relBuilder.getTypeFactory)
-    val newTableSource = newRelOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource
-    val oldTableSource = relOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource
+    val newTableSource = newRelOptTable.unwrap(classOf[LegacyTableSourceTable[_]]).tableSource
+    val oldTableSource = relOptTable.unwrap(classOf[LegacyTableSourceTable[_]]).tableSource
 
     if (newTableSource.asInstanceOf[FilterableTableSource[_]].isFilterPushedDown
       && newTableSource.explainSource().equals(oldTableSource.explainSource)) {
@@ -137,7 +137,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
       relOptTable: FlinkPreparingTableBase,
       typeFactory: RelDataTypeFactory): FlinkPreparingTableBase = {
     val originPredicatesSize = predicates.size()
-    val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = relOptTable.unwrap(classOf[LegacyTableSourceTable[_]])
     val filterableSource = tableSourceTable.tableSource.asInstanceOf[FilterableTableSource[_]]
     val newTableSource = filterableSource.applyPredicate(predicates)
     val updatedPredicatesSize = predicates.size()
@@ -157,6 +157,6 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
 }
 
 
-object PushFilterIntoTableSourceScanRule {
-  val INSTANCE: RelOptRule = new PushFilterIntoTableSourceScanRule
+object PushFilterIntoLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushFilterIntoLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRule.scala
similarity index 81%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRule.scala
index e58240a..4ffd98c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushLimitIntoLegacyTableSourceScanRule.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.planner.plan.rules.logical
 
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.plan.stats.TableStats
-import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalSort, FlinkLogicalTableSourceScan}
-import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, TableSourceTable}
+import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalSort, FlinkLogicalLegacyTableSourceScan}
+import org.apache.flink.table.planner.plan.schema.{FlinkPreparingTableBase, LegacyTableSourceTable}
 import org.apache.flink.table.planner.plan.stats.FlinkStatistic
 import org.apache.flink.table.sources.LimitableTableSource
 
@@ -46,17 +46,18 @@ import java.util.Collections
   * implement.
   * 3.We can support limit with offset, we can push down offset + fetch to table source.
   */
-class PushLimitIntoTableSourceScanRule extends RelOptRule(
+class PushLimitIntoLegacyTableSourceScanRule extends RelOptRule(
   operand(classOf[FlinkLogicalSort],
-    operand(classOf[FlinkLogicalTableSourceScan], none)), "PushLimitIntoTableSourceScanRule") {
+    operand(classOf[FlinkLogicalLegacyTableSourceScan], none)),
+  "PushLimitIntoLegacyTableSourceScanRule") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val sort = call.rel(0).asInstanceOf[Sort]
     val onlyLimit = sort.getCollation.getFieldCollations.isEmpty && sort.fetch != null
     if (onlyLimit) {
       call.rel(1).asInstanceOf[TableScan]
-          .getTable.unwrap(classOf[TableSourceTable[_]]) match {
-        case table: TableSourceTable[_] =>
+          .getTable.unwrap(classOf[LegacyTableSourceTable[_]]) match {
+        case table: LegacyTableSourceTable[_] =>
           table.tableSource match {
             case source: LimitableTableSource[_] =>
               return !source.isLimitPushedDown
@@ -70,16 +71,16 @@ class PushLimitIntoTableSourceScanRule extends RelOptRule(
 
   override def onMatch(call: RelOptRuleCall): Unit = {
     val sort = call.rel(0).asInstanceOf[Sort]
-    val scan = call.rel(1).asInstanceOf[FlinkLogicalTableSourceScan]
-    val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val scan = call.rel(1).asInstanceOf[FlinkLogicalLegacyTableSourceScan]
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     val offset = if (sort.offset == null) 0 else RexLiteral.intValue(sort.offset)
     val limit = offset + RexLiteral.intValue(sort.fetch)
     val relBuilder = call.builder()
     val newRelOptTable = applyLimit(limit, tableSourceTable, relBuilder)
     val newScan = scan.copy(scan.getTraitSet, newRelOptTable)
 
-    val newTableSource = newRelOptTable.unwrap(classOf[TableSourceTable[_]]).tableSource
-    val oldTableSource = tableSourceTable.unwrap(classOf[TableSourceTable[_]]).tableSource
+    val newTableSource = newRelOptTable.unwrap(classOf[LegacyTableSourceTable[_]]).tableSource
+    val oldTableSource = tableSourceTable.unwrap(classOf[LegacyTableSourceTable[_]]).tableSource
 
     if (newTableSource.asInstanceOf[LimitableTableSource[_]].isLimitPushedDown
         && newTableSource.explainSource().equals(oldTableSource.explainSource)) {
@@ -94,8 +95,8 @@ class PushLimitIntoTableSourceScanRule extends RelOptRule(
   private def applyLimit(
       limit: Long,
       relOptTable: FlinkPreparingTableBase,
-      relBuilder: RelBuilder): TableSourceTable[_] = {
-    val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[Any]])
+      relBuilder: RelBuilder): LegacyTableSourceTable[_] = {
+    val tableSourceTable = relOptTable.unwrap(classOf[LegacyTableSourceTable[Any]])
     val limitedSource = tableSourceTable.tableSource.asInstanceOf[LimitableTableSource[Any]]
     val newTableSource = limitedSource.applyLimit(limit)
 
@@ -115,6 +116,6 @@ class PushLimitIntoTableSourceScanRule extends RelOptRule(
   }
 }
 
-object PushLimitIntoTableSourceScanRule {
-  val INSTANCE: RelOptRule = new PushLimitIntoTableSourceScanRule
+object PushLimitIntoLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushLimitIntoLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRule.scala
similarity index 95%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRule.scala
index 63117d7..3ff3384 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoLegacyTableSourceScanRule.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.catalog.{Catalog, CatalogPartitionSpec, ObjectIden
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.plan.stats.TableStats
 import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkTypeFactory}
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.planner.plan.stats.FlinkStatistic
 import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, PartitionPruner, RexNodeExtractor, RexNodeToExpressionConverter}
 import org.apache.flink.table.planner.utils.CatalogTableStatisticsConverter
@@ -47,10 +47,10 @@ import scala.collection.mutable
   * Planner rule that tries to push partitions evaluated by filter condition into a
   * [[PartitionableTableSource]].
   */
-class PushPartitionIntoTableSourceScanRule extends RelOptRule(
+class PushPartitionIntoLegacyTableSourceScanRule extends RelOptRule(
   operand(classOf[Filter],
     operand(classOf[LogicalTableScan], none)),
-  "PushPartitionIntoTableSourceScanRule") {
+  "PushPartitionIntoLegacyTableSourceScanRule") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val filter: Filter = call.rel(0)
@@ -59,8 +59,8 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule(
     }
 
     val scan: LogicalTableScan = call.rel(1)
-    scan.getTable.unwrap(classOf[TableSourceTable[_]]) match {
-      case table: TableSourceTable[_] =>
+    scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]]) match {
+      case table: LegacyTableSourceTable[_] =>
         table.catalogTable.isPartitioned &&
           table.tableSource.isInstanceOf[PartitionableTableSource]
       case _ => false
@@ -72,7 +72,7 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule(
     val scan: LogicalTableScan = call.rel(1)
     val context = call.getPlanner.getContext.unwrap(classOf[FlinkContext])
     val config = context.getTableConfig
-    val tableSourceTable: TableSourceTable[_] = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     val tableIdentifier = tableSourceTable.tableIdentifier
     val catalogOption = toScala(context.getCatalogManager.getCatalog(
       tableIdentifier.getCatalogName))
@@ -262,6 +262,6 @@ class PushPartitionIntoTableSourceScanRule extends RelOptRule(
   }
 }
 
-object PushPartitionIntoTableSourceScanRule {
-  val INSTANCE: RelOptRule = new PushPartitionIntoTableSourceScanRule
+object PushPartitionIntoLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushPartitionIntoLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRule.scala
similarity index 90%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRule.scala
index a553fa2..455740d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRule.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.planner.plan.rules.logical
 
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.planner.plan.utils._
 import org.apache.flink.table.sources._
 import org.apache.flink.util.CollectionUtil
@@ -33,15 +33,15 @@ import org.apache.flink.table.api.TableException
   * Planner rule that pushes a [[LogicalProject]] into a [[LogicalTableScan]]
   * which wraps a [[ProjectableTableSource]] or a [[NestedFieldsProjectableTableSource]].
   */
-class PushProjectIntoTableSourceScanRule extends RelOptRule(
+class PushProjectIntoLegacyTableSourceScanRule extends RelOptRule(
   operand(classOf[LogicalProject],
     operand(classOf[LogicalTableScan], none)),
-  "PushProjectIntoTableSourceScanRule") {
+  "PushProjectIntoLegacyTableSourceScanRule") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val scan: LogicalTableScan = call.rel(1)
-    scan.getTable.unwrap(classOf[TableSourceTable[_]]) match {
-      case table: TableSourceTable[_] =>
+    scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]]) match {
+      case table: LegacyTableSourceTable[_] =>
         table.tableSource match {
           // projection pushdown is not supported for sources that provide time indicators
           case r: DefinedRowtimeAttributes if !CollectionUtil.isNullOrEmpty(
@@ -65,7 +65,7 @@ class PushProjectIntoTableSourceScanRule extends RelOptRule(
       return
     }
 
-    val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     val oldTableSource = tableSourceTable.tableSource
     val (newTableSource, isProjectSuccess) = oldTableSource match {
       case nested: NestedFieldsProjectableTableSource[_] =>
@@ -117,6 +117,6 @@ class PushProjectIntoTableSourceScanRule extends RelOptRule(
   }
 }
 
-object PushProjectIntoTableSourceScanRule {
-  val INSTANCE: RelOptRule = new PushProjectIntoTableSourceScanRule
+object PushProjectIntoLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushProjectIntoLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecLegacyTableSourceScanRule.scala
similarity index 71%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecLegacyTableSourceScanRule.scala
index 857994a..75682b3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecLegacyTableSourceScanRule.scala
@@ -19,9 +19,9 @@
 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.FlinkLogicalTableSourceScan
-import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecTableSourceScan
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.sources.StreamTableSource
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
@@ -30,21 +30,21 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.calcite.rel.core.TableScan
 
 /**
-  * Rule that converts [[FlinkLogicalTableSourceScan]] to [[BatchExecTableSourceScan]].
+  * Rule that converts [[FlinkLogicalLegacyTableSourceScan]] to [[BatchExecLegacyTableSourceScan]].
   */
-class BatchExecTableSourceScanRule
+class BatchExecLegacyTableSourceScanRule
   extends ConverterRule(
-    classOf[FlinkLogicalTableSourceScan],
+    classOf[FlinkLogicalLegacyTableSourceScan],
     FlinkConventions.LOGICAL,
     FlinkConventions.BATCH_PHYSICAL,
-    "BatchExecScanTableSourceRule") {
+    "BatchExecLegacyTableSourceScanRule") {
 
   /** Rule must only match if TableScan targets a bounded [[StreamTableSource]] */
   override def matches(call: RelOptRuleCall): Boolean = {
     val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     tableSourceTable match {
-      case tst: TableSourceTable[_] =>
+      case tst: LegacyTableSourceTable[_] =>
         tst.tableSource match {
           case sts: StreamTableSource[_] => sts.isBounded
           case _ => false
@@ -54,16 +54,16 @@ class BatchExecTableSourceScanRule
   }
 
   def convert(rel: RelNode): RelNode = {
-    val scan = rel.asInstanceOf[FlinkLogicalTableSourceScan]
+    val scan = rel.asInstanceOf[FlinkLogicalLegacyTableSourceScan]
     val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
-    new BatchExecTableSourceScan(
+    new BatchExecLegacyTableSourceScan(
       rel.getCluster,
       newTrait,
-      scan.getTable.asInstanceOf[TableSourceTable[_]]
+      scan.getTable.asInstanceOf[LegacyTableSourceTable[_]]
     )
   }
 }
 
-object BatchExecTableSourceScanRule {
-  val INSTANCE: RelOptRule = new BatchExecTableSourceScanRule
+object BatchExecLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new BatchExecLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/common/CommonLookupJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/common/CommonLookupJoinRule.scala
index 62bd3dd..70e1324 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/common/CommonLookupJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/common/CommonLookupJoinRule.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.common
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.planner.plan.nodes.common.CommonLookupJoin
 import org.apache.flink.table.planner.plan.nodes.logical._
-import org.apache.flink.table.planner.plan.nodes.physical.PhysicalTableSourceScan
+import org.apache.flink.table.planner.plan.nodes.physical.PhysicalLegacyTableSourceScan
 import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType
 import org.apache.flink.table.planner.plan.utils.JoinUtil
 import org.apache.flink.table.sources.{LookupableTableSource, TableSource}
@@ -73,8 +73,8 @@ trait CommonLookupJoinRule {
 
   protected def findTableSource(relNode: RelNode): Option[TableSource[_]] = {
     relNode match {
-      case logicalScan: FlinkLogicalTableSourceScan => Some(logicalScan.tableSource)
-      case physicalScan: PhysicalTableSourceScan => Some(physicalScan.tableSource)
+      case logicalScan: FlinkLogicalLegacyTableSourceScan => Some(logicalScan.tableSource)
+      case physicalScan: PhysicalLegacyTableSourceScan => Some(physicalScan.tableSource)
       // TODO: find TableSource in FlinkLogicalIntermediateTableScan
       case _ => None
     }
@@ -82,9 +82,9 @@ trait CommonLookupJoinRule {
 
   protected def isLookupableTableSource(relNode: RelNode): Boolean = {
     relNode match {
-      case logicalScan: FlinkLogicalTableSourceScan =>
+      case logicalScan: FlinkLogicalLegacyTableSourceScan =>
         logicalScan.tableSource.isInstanceOf[LookupableTableSource[_]]
-      case physicalScan: PhysicalTableSourceScan =>
+      case physicalScan: PhysicalLegacyTableSourceScan =>
         physicalScan.tableSource.isInstanceOf[LookupableTableSource[_]]
       // TODO: find TableSource in FlinkLogicalIntermediateTableScan
       case _ => false
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
index 9f66a2a..3971744 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.stream
 
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.planner.plan.`trait`.{MiniBatchInterval, MiniBatchIntervalTrait, MiniBatchIntervalTraitDef, MiniBatchMode}
-import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamExecDataStreamScan, StreamExecGroupWindowAggregate, StreamExecMiniBatchAssigner, StreamExecTableSourceScan, StreamExecWatermarkAssigner, StreamPhysicalRel}
+import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamExecDataStreamScan, StreamExecGroupWindowAggregate, StreamExecMiniBatchAssigner, StreamExecLegacyTableSourceScan, StreamExecWatermarkAssigner, StreamPhysicalRel}
 import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil
 import org.apache.calcite.plan.RelOptRule._
 import org.apache.calcite.plan.hep.HepRelVertex
@@ -122,7 +122,7 @@ class MiniBatchIntervalInferRule extends RelOptRule(
       .getMiniBatchInterval
       .mode
     node match {
-      case _: StreamExecDataStreamScan | _: StreamExecTableSourceScan =>
+      case _: StreamExecDataStreamScan | _: StreamExecLegacyTableSourceScan =>
         // append minibatch node if the mode is not NONE and reach a source leaf node
         mode == MiniBatchMode.RowTime || mode == MiniBatchMode.ProcTime
       case _: StreamExecWatermarkAssigner  =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecLegacyTableSourceScanRule.scala
similarity index 70%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecLegacyTableSourceScanRule.scala
index 6b785b3..88fd048 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecLegacyTableSourceScanRule.scala
@@ -19,9 +19,9 @@
 package org.apache.flink.table.planner.plan.rules.physical.stream
 
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
-import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableSourceScan
-import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecTableSourceScan
-import org.apache.flink.table.planner.plan.schema.TableSourceTable
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamExecLegacyTableSourceScan
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable
 import org.apache.flink.table.sources.StreamTableSource
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
@@ -30,21 +30,21 @@ import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.calcite.rel.core.TableScan
 
 /**
-  * Rule that converts [[FlinkLogicalTableSourceScan]] to [[StreamExecTableSourceScan]].
+  * Rule that converts [[FlinkLogicalLegacyTableSourceScan]] to [[StreamExecLegacyTableSourceScan]].
   */
-class StreamExecTableSourceScanRule
+class StreamExecLegacyTableSourceScanRule
   extends ConverterRule(
-    classOf[FlinkLogicalTableSourceScan],
+    classOf[FlinkLogicalLegacyTableSourceScan],
     FlinkConventions.LOGICAL,
     FlinkConventions.STREAM_PHYSICAL,
-    "StreamExecTableSourceScanRule") {
+    "StreamExecLegacyTableSourceScanRule") {
 
   /** Rule must only match if TableScan targets a [[StreamTableSource]] */
   override def matches(call: RelOptRuleCall): Boolean = {
     val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]])
+    val tableSourceTable = scan.getTable.unwrap(classOf[LegacyTableSourceTable[_]])
     tableSourceTable match {
-      case tst: TableSourceTable[_] =>
+      case tst: LegacyTableSourceTable[_] =>
         tst.tableSource match {
           case _: StreamTableSource[_] => true
           case _ => false
@@ -54,17 +54,17 @@ class StreamExecTableSourceScanRule
   }
 
   def convert(rel: RelNode): RelNode = {
-    val scan: FlinkLogicalTableSourceScan = rel.asInstanceOf[FlinkLogicalTableSourceScan]
+    val scan: FlinkLogicalLegacyTableSourceScan = rel.asInstanceOf[FlinkLogicalLegacyTableSourceScan]
     val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL)
 
-    new StreamExecTableSourceScan(
+    new StreamExecLegacyTableSourceScan(
       rel.getCluster,
       traitSet,
-      scan.getTable.asInstanceOf[TableSourceTable[_]]
+      scan.getTable.asInstanceOf[LegacyTableSourceTable[_]]
     )
   }
 }
 
-object StreamExecTableSourceScanRule {
-  val INSTANCE: RelOptRule = new StreamExecTableSourceScanRule
+object StreamExecLegacyTableSourceScanRule {
+  val INSTANCE: RelOptRule = new StreamExecLegacyTableSourceScanRule
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala
index 83a7f59..a03ee13 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.scala
@@ -42,9 +42,9 @@ import scala.collection.JavaConverters._
 
 /**
   * A [[FlinkPreparingTableBase]] implementation which defines the interfaces required to translate
-  * the Calcite [[RelOptTable]] to the Flink specific [[TableSourceTable]].
+  * the Calcite [[RelOptTable]] to the Flink specific [[LegacyTableSourceTable]].
   *
-  * <p>This table is only used to translate the catalog table into [[TableSourceTable]]
+  * <p>This table is only used to translate the catalog table into [[LegacyTableSourceTable]]
   * during the last phrase of sql-to-rel conversion, it is overdue once the sql node was converted
   * to relational expression.
   *
@@ -87,10 +87,10 @@ class CatalogSourceTable[T](
         + s"is set to true")
     }
 
-    val tableSource = findAndCreateTableSource(
+    val tableSource = findAndCreateLegacyTableSource(
       hintedOptions,
       conf)
-    val tableSourceTable = new TableSourceTable[T](
+    val tableSourceTable = new LegacyTableSourceTable[T](
       relOptSchema,
       schemaTable.getTableIdentifier,
       erasedRowType,
@@ -159,8 +159,8 @@ class CatalogSourceTable[T](
     relBuilder.build()
   }
 
-  /** Create the table source. */
-  private def findAndCreateTableSource(
+  /** Create the legacy table source. */
+  private def findAndCreateLegacyTableSource(
       hintedOptions: JMap[String, String],
       conf: ReadableConfig): TableSource[T] = {
     val tableFactoryOpt = schemaTable.getTableFactory
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyTableSourceTable.scala
similarity index 96%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyTableSourceTable.scala
index e138e06..ba8bf9e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyTableSourceTable.scala
@@ -46,7 +46,7 @@ import scala.collection.JavaConverters._
   * @param isStreamingMode A flag that tells if the current table is in stream mode
   * @param catalogTable Catalog table where this table source table comes from
   */
-class TableSourceTable[T](
+class LegacyTableSourceTable[T](
     relOptSchema: RelOptSchema,
     val tableIdentifier: ObjectIdentifier,
     rowType: RelDataType,
@@ -99,8 +99,8 @@ class TableSourceTable[T](
     * @param statistic New FlinkStatistic to replace
     * @return New TableSourceTable instance with specified table source and [[FlinkStatistic]]
     */
-  def copy(tableSource: TableSource[_], statistic: FlinkStatistic): TableSourceTable[T] = {
-    new TableSourceTable[T](
+  def copy(tableSource: TableSource[_], statistic: FlinkStatistic): LegacyTableSourceTable[T] = {
+    new LegacyTableSourceTable[T](
       relOptSchema,
       tableIdentifier,
       rowType,
@@ -120,7 +120,7 @@ class TableSourceTable[T](
     * @return New TableSourceTable instance with specified table source
     *         and selected fields
     */
-  def copy(tableSource: TableSource[_], selectedFields: Array[Int]): TableSourceTable[T] = {
+  def copy(tableSource: TableSource[_], selectedFields: Array[Int]): LegacyTableSourceTable[T] = {
     val newRowType = relOptSchema
       .getTypeFactory
       .createStructType(
@@ -128,7 +128,7 @@ class TableSourceTable[T](
           .map(idx => rowType.getFieldList.get(idx))
           .toList
           .asJava)
-    new TableSourceTable[T](
+    new LegacyTableSourceTable[T](
       relOptSchema,
       tableIdentifier,
       newRowType,
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out
index 29017e0..3ac5cf1 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out
+++ b/flink-table/flink-table-planner-blink/src/test/resources/explain/testSqlUpdateAndToDataStream.out
@@ -5,7 +5,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first
 
 == Optimized Logical Plan ==
 Sink(name=[`default_catalog`.`default_database`.`MySink1`], fields=[first])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
 
 == Physical Execution Plan ==
  : Data Source
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out b/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out
index b0758cb..454d103 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out
+++ b/flink-table/flink-table-planner-blink/src/test/resources/explain/testStreamTableEnvironmentExplain.out
@@ -5,7 +5,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first]
 
 == Optimized Logical Plan ==
 Sink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
 
 == Physical Execution Plan ==
  : Data Source
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
index 7189eb2..e464dfb 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
@@ -581,7 +581,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 == Physical Execution Plan ==
  : Data Source
@@ -601,7 +601,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
 
 == Physical Execution Plan ==
  : Data Source
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
index 2ceb4c1..dc5dde8 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
@@ -101,7 +101,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 == Physical Execution Plan ==
  : Data Source
@@ -121,7 +121,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], changelogMode=[I]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
 
 == Physical Execution Plan ==
  : Data Source
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/executor/BatchExecutorTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/executor/BatchExecutorTest.xml
index 52a0f64..1eff1f4 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/executor/BatchExecutorTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/executor/BatchExecutorTest.xml
@@ -23,7 +23,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 == Physical Execution Plan ==
  : Data Source
@@ -43,7 +43,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
 +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]])
 
 == Optimized Logical Plan ==
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 == Physical Execution Plan ==
  : Data Source
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml
index 09e443a..eac1d71 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml
@@ -29,7 +29,7 @@ LogicalProject(EXPR$0=[ARRAY(_UTF-16LE'Hi', _UTF-16LE'Hello', $2)])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[ARRAY(_UTF-16LE'Hi', _UTF-16LE'Hello', c) AS EXPR$0])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -64,7 +64,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c], where=[AND(<(a, 10), >(b, 20))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -82,7 +82,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c], where=[OR(<(a, 10), >(a, 20))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -103,7 +103,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, CAST(1984-07-12) AS b, CAST(14:34:24) AS c, CAST(1984-07-12 14:34:24:TIMESTAMP(3)) AS d], where=[AND(=(b, 1984-07-12), =(c, 14:34:24), =(d, 1984-07-12 14:34:24:TIMESTAMP(3)))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -121,7 +121,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, CAST(_UTF-16LE'xx':VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS c], where=[AND(IN(b, 1, 3, 4, 5, 6), =(c, _UTF-16LE'xx':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -139,7 +139,7 @@ LogicalProject(EXPR$0=[ROW($0, $1, $2)], EXPR$1=[ARRAY(12, $1)], EXPR$2=[MAP($0,
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[ROW(a, b, c) AS EXPR$0, ARRAY(12, b) AS EXPR$1, MAP(a, c) AS EXPR$2], where=[=(ROW(a, b, c), CAST(ROW(_UTF-16LE'foo', 12, 1984-07-12 14:34:24)))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable5, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable5, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -159,7 +159,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c], where=[AND(>(b, 0), <(a, 50))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -176,7 +176,7 @@ LogicalProject(_1=[$0._1], _2=[$0._2], c=[$2], _10=[$1._1], _20=[$1._2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a._1 AS _1, a._2 AS _2, c, b._1 AS _10, b._2 AS _20])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -193,7 +193,7 @@ LogicalProject(c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[c])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -210,7 +210,7 @@ LogicalProject(EXPR$0=[MAP($0, $2)])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[MAP(a, c) AS EXPR$0])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -228,7 +228,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c], where=[OR(NOT IN(b, 1, 3, 4, 5, 6), =(c, _UTF-16LE'xx':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"))])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -246,7 +246,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c], where=[>(b, 0)])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -263,7 +263,7 @@ LogicalProject(a=[$0], c=[$2])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, c])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -279,7 +279,7 @@ LogicalProject(a=[$0])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable4, source: [TestTableSource(a)]]], fields=[a])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable4, source: [TestTableSource(a)]]], fields=[a])
 ]]>
     </Resource>
   </TestCase>
@@ -296,7 +296,7 @@ LogicalProject(EXPR$0=[MAP($1, 30, 10, $0)])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[MAP(b, 30, 10, a) AS EXPR$0])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -314,7 +314,7 @@ LogicalProject(a=[$0], EXPR$1=[+($1, 1)])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, +(b, 1) AS EXPR$1], where=[>(b, 2)])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -337,7 +337,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], EXPR$3=[1984-07-12], EXPR$4=[14:34:24], E
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b, c, 1984-07-12 AS EXPR$3, 14:34:24 AS EXPR$4, 1984-07-12 14:34:24 AS EXPR$5])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -353,7 +353,7 @@ LogicalProject(1-_./Ü=[$0], b=[$1], c=[$2])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -370,7 +370,7 @@ LogicalProject(EXPR$0=[ROW(1, _UTF-16LE'Hi', $0)])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[ROW(1, _UTF-16LE'Hi', a) AS EXPR$0])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml
index f48686c..dd9eac2 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.xml
@@ -53,7 +53,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[2])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[2])
 
 Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], reuse_id=[1])
 +- Reused(reference_id=[2])
@@ -105,7 +105,7 @@ Calc(select=[sum_a], reuse_id=[1])
    +- Exchange(distribution=[hash[c]])
       +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0])
          +- Calc(select=[c, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -154,7 +154,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, b1])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a, b], where=[<=(a, 10)], reuse_id=[1])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Union(all=[true], union=[a, b], reuse_id=[2])
 :- Reused(reference_id=[1])
@@ -164,7 +164,7 @@ Union(all=[true], union=[a, b], reuse_id=[2])
       :  +- Reused(reference_id=[1])
       +- Exchange(distribution=[hash[a]])
          +- Calc(select=[a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b1])
 +- Reused(reference_id=[2])
@@ -204,14 +204,14 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, b1])
       <![CDATA[
 Calc(select=[a AS a1, b AS b1], reuse_id=[1])
 +- Calc(select=[a, b, c], where=[<=(a, 10)])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Calc(select=[a, b1], reuse_id=[2])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, a1)], select=[a1, b1, a, b, c, d, e], build=[left])
    :- Exchange(distribution=[hash[a1]])
    :  +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[a]])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b1])
 +- Reused(reference_id=[2])
@@ -263,7 +263,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b,
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
 
 Calc(select=[a, b, c], where=[>=(a, 0)], reuse_id=[2])
 +- Reused(reference_id=[1])
@@ -309,7 +309,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_m
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[random_udf(a) AS a], reuse_id=[1])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -352,10 +352,10 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_m
 Union(all=[true], union=[a], reuse_id=[1])
 :- Calc(select=[a])
 :  +- Calc(select=[a, c])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[d AS a])
    +- Calc(select=[d, f])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -412,15 +412,15 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink3`], fields=[a])
 Union(all=[true], union=[a], reuse_id=[2])
 :- Calc(select=[a])
 :  +- Calc(select=[a, c])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[d AS a])
    +- Calc(select=[d, f])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 
 Union(all=[true], union=[a], reuse_id=[1])
 :- Reused(reference_id=[2])
 +- Calc(select=[a])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -480,9 +480,9 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink3`], fields=[total_m
       <![CDATA[
 Union(all=[true], union=[a, c], reuse_id=[1])
 :- Calc(select=[a, c])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[d, f])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, c])
 +- Reused(reference_id=[1])
@@ -492,7 +492,7 @@ Union(all=[true], union=[a], reuse_id=[2])
 :  +- Reused(reference_id=[1])
 +- Calc(select=[a])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -542,13 +542,13 @@ Union(all=[true], union=[a], reuse_id=[1])
 :- Union(all=[true], union=[a])
 :  :- Calc(select=[a])
 :  :  +- Calc(select=[a, c])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Calc(select=[d AS a])
 :     +- Calc(select=[d, f])
-:        +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Calc(select=[a])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[total_sum])
 +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
@@ -586,7 +586,7 @@ HashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 15000)], p
 +- Exchange(distribution=[hash[a]])
    +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 15000)], properties=[w$start, w$end, w$rowtime], select=[a, Partial_SUM(c) AS sum$0])
       +- Calc(select=[ts, a, CAST(c) AS c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]], fields=[a, b, c, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, ts)]]], fields=[a, b, c, ts])
 
 Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, sum_c, time, window_start, window_end])
 +- Calc(select=[a, sum_c, w$end AS time, w$start AS window_start, w$end AS window_end])
@@ -638,7 +638,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, tota
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
 
 Calc(select=[a, total_c], reuse_id=[2])
 +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b, Final_COUNT(count$0) AS total_c])
@@ -687,7 +687,7 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[c, cnt])
    +- Exchange(distribution=[hash[c]])
       +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(a) AS count$0])
          +- Calc(select=[c, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -723,7 +723,7 @@ Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b, c1
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[right])
       :- Exchange(distribution=[hash[a3]], shuffle_mode=[BATCH])
       :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
       +- Calc(select=[a AS a1, b])
          +- HashJoin(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2], build=[right])
             :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
@@ -757,7 +757,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1
 Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a1, b1])
 +- Union(all=[true], union=[a, b])
    :- Calc(select=[a, b], where=[<=(a, 10)], reuse_id=[1])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Calc(select=[a0 AS a, b AS b1])
       +- HashJoin(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], build=[left])
          :- Exchange(distribution=[hash[a]])
@@ -806,7 +806,7 @@ Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, a3)], select=[a, a3, b1, c1], build=[right])
       :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
       :  +- Calc(select=[a], where=[<=(a, 10)])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
       +- Calc(select=[a3, b AS b1, c1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[right])
             :- Exchange(distribution=[hash[a3]], shuffle_mode=[BATCH])
@@ -847,11 +847,11 @@ Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c, d, e,
       :- Exchange(distribution=[hash[a]])
       :  +- HashJoin(joinType=[InnerJoin], where=[=(b, e)], select=[a, b, c, d, e, f], build=[right])
       :     :- Exchange(distribution=[hash[b]])
-      :     :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       :     +- Exchange(distribution=[hash[e]])
-      :        +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
       +- Exchange(distribution=[hash[i]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m])
 ]]>
     </Resource>
   </TestCase>
@@ -876,9 +876,9 @@ Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[total_sum])
       +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
          +- Union(all=[true], union=[a])
             :- Calc(select=[a])
-            :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
             +- Calc(select=[d AS a])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
index 3bbf8bc..ca616e9 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
@@ -119,7 +119,7 @@ Calc(select=[c, a, b, c0, a1, b0])
    :     +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b], reuse_id=[1])
    :        +- Exchange(distribution=[hash[c]])
    :           +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[c, a, b], where=[>(a, 1)])
          +- Reused(reference_id=[1])
@@ -169,7 +169,7 @@ HashJoin(joinType=[InnerJoin], where=[=(c, c1)], select=[a, b, c, a0, b0, c0, a1
 :     :        +- Exchange(distribution=[hash[a]])
 :     :           +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_MAX(c) AS max$1])
 :     :              +- Sort(orderBy=[a ASC])
-:     :                 +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
 :        +- Reused(reference_id=[1])
 +- Exchange(distribution=[hash[c]])
@@ -213,7 +213,7 @@ Calc(select=[a, b])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, a0, b], build=[right])
    :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
    :  +- Calc(select=[a], where=[=(b, 5:BIGINT)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
    +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, a0)], select=[a, b], build=[left])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b])
@@ -256,7 +256,7 @@ Calc(select=[a])
    :     +- Limit(offset=[0], fetch=[10], global=[true])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Reused(reference_id=[1])
 ]]>
@@ -291,7 +291,7 @@ Calc(select=[a])
    :     +- Limit(offset=[0], fetch=[10], global=[true])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[any], shuffle_mode=[BATCH])
       +- Reused(reference_id=[1])
 ]]>
@@ -328,10 +328,10 @@ HashJoin(joinType=[InnerJoin], where=[=(b, e0)], select=[a, b, d, e, a0, b0, d0,
 :  +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d, e], build=[left])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[<(a, 10)])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
 :     +- Exchange(distribution=[hash[d]], reuse_id=[2])
 :        +- Calc(select=[d, e])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[e]])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d, e], build=[left])
       :- Exchange(distribution=[hash[a]])
@@ -374,7 +374,7 @@ HashJoin(joinType=[InnerJoin], where=[=(c, c0)], select=[a, b, c, a0, b0, c0], b
 :           +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1])
 :              +- Sort(orderBy=[b ASC])
 :                 +- Exchange(distribution=[hash[b]])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[c]])
    +- Calc(select=[w0$o0 AS a, b, c])
       +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MIN($2) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[b, c, $2, w0$o0])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml
index 4456145..656232e 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/LimitTest.xml
@@ -50,7 +50,7 @@ Calc(select=[a, c])
 +- Limit(offset=[0], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[10], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 10]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 10]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -71,7 +71,7 @@ Calc(select=[a, c])
 +- Limit(offset=[10], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[20], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -92,7 +92,7 @@ Calc(select=[a, c])
 +- Limit(offset=[10], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[20], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 20]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 20]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -113,7 +113,7 @@ Calc(select=[a, c])
 +- Limit(offset=[0], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[10], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -185,7 +185,7 @@ Calc(select=[a, c])
 +- Limit(offset=[0], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[10], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 10]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 10]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -206,7 +206,7 @@ Calc(select=[a, c])
 +- SortLimit(orderBy=[c ASC], offset=[0], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- SortLimit(orderBy=[c ASC], offset=[0], fetch=[10], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, LimitTable]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -227,7 +227,7 @@ Calc(select=[a, c])
 +- Limit(offset=[1], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[11], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -248,7 +248,7 @@ Calc(select=[a, c])
 +- Limit(offset=[0], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[10], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -268,7 +268,7 @@ LogicalSort(fetch=[5])
 Limit(offset=[0], fetch=[5], global=[true])
 +- Exchange(distribution=[single])
    +- Limit(offset=[0], fetch=[5], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -288,7 +288,7 @@ LogicalSort(offset=[10])
 Calc(select=[a, c])
 +- Limit(offset=[10], fetch=[unlimited], global=[true])
    +- Exchange(distribution=[single])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -309,7 +309,7 @@ Calc(select=[a, c])
 +- Limit(offset=[1], fetch=[10], global=[true])
    +- Exchange(distribution=[single])
       +- Limit(offset=[0], fetch=[11], global=[false])
-         +- TableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 11]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, LimitTable, source: [limit: 11]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml
index 3ba1418..994c23e 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.xml
@@ -31,7 +31,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]
       <![CDATA[
 Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c])
 +- Sort(orderBy=[b ASC, c ASC])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -51,7 +51,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sinkShuffleBy`], fields=
 Sink(name=[`default_catalog`.`default_database`.`sinkShuffleBy`], fields=[a, b, c])
 +- Sort(orderBy=[b ASC, c ASC])
    +- Exchange(distribution=[hash[b, c]])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -71,7 +71,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]
 Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c])
 +- Sort(orderBy=[c ASC])
    +- Calc(select=[a, 1:BIGINT AS EXPR$1, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -90,7 +90,7 @@ LogicalSink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]
       <![CDATA[
 Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c])
 +- Calc(select=[a, 1:BIGINT AS EXPR$1, 1:BIGINT AS EXPR$2])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml
index 05bdc9e..b1cc4aa 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RankTest.xml
@@ -40,7 +40,7 @@ Calc(select=[a, b, $2])
       +- Exchange(distribution=[hash[b]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[b], orderBy=[a ASC], global=[false], select=[a, b, rk])
             +- Sort(orderBy=[b ASC, a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, rk)]]], fields=[a, b, rk])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, rk)]]], fields=[a, b, rk])
 ]]>
     </Resource>
   </TestCase>
@@ -70,7 +70,7 @@ Calc(select=[a, b, rk2 AS rk1, rk2])
       +- Exchange(distribution=[hash[b]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[b], orderBy=[a ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[b ASC, a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -98,7 +98,7 @@ Calc(select=[a, rk, b, c])
       +- Exchange(distribution=[hash[a]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[a], orderBy=[a ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -126,7 +126,7 @@ Calc(select=[a, b, 2:BIGINT AS rk])
       +- Exchange(distribution=[hash[b]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC, c ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[b ASC, a ASC, c ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -154,7 +154,7 @@ Calc(select=[a, b, $2])
       +- Exchange(distribution=[hash[b, c]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b, c], orderBy=[a ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[b ASC, c ASC, a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -182,7 +182,7 @@ Calc(select=[a, b, $2], where=[>(a, 10)])
       +- Exchange(distribution=[hash[b]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=2], partitionBy=[b], orderBy=[a ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[b ASC, a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -210,7 +210,7 @@ Calc(select=[a, b, $2])
       +- Exchange(distribution=[single])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=9], partitionBy=[], orderBy=[a ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[a ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.xml
index 401483c..c4ad1e0 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.xml
@@ -42,10 +42,10 @@ Calc(select=[EXPR$0])
       +- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d])
          :- Exchange(distribution=[hash[a]])
          :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
          +- Exchange(distribution=[hash[d]])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -75,10 +75,10 @@ Calc(select=[EXPR$0])
       +- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d])
          :- Exchange(distribution=[hash[a]])
          :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
          +- Exchange(distribution=[hash[d]])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -103,7 +103,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b, Final_COUNT(count$0)
    +- Exchange(distribution=[hash[a, b]])
       +- LocalSortAggregate(groupBy=[a, b], select=[a, b, Partial_COUNT(c) AS count$0])
          +- Sort(orderBy=[a ASC, b ASC])
-            +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -133,7 +133,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f1=[$3], f=[$4], cnt=[$5])
       <![CDATA[
 SortMergeJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f1, f, cnt], rightSorted=[true])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[f0 AS f1, f, cnt])
    +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
       +- SortAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
@@ -142,7 +142,7 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f1, f, cnt
                +- LocalSortAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                   +- Sort(orderBy=[f ASC])
                      +- Calc(select=[f])
-                        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -172,7 +172,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f=[$3], f1=[$4])
       <![CDATA[
 SortMergeJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f, f1], rightSorted=[true])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[f, f0 AS f1])
    +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
       +- SortAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
@@ -181,7 +181,7 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f, f1], ri
                +- LocalSortAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                   +- Sort(orderBy=[f ASC])
                      +- Calc(select=[f], where=[LIKE(f, _UTF-16LE'%llo%')])
-                        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -211,7 +211,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f1=[$3])
       <![CDATA[
 SortMergeJoin(joinType=[InnerJoin], where=[=(c, f1)], select=[a, b, c, f1])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f1]])
    +- Calc(select=[f0 AS f1])
       +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
@@ -221,7 +221,7 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(c, f1)], select=[a, b, c, f1])
                   +- LocalSortAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                      +- Sort(orderBy=[f ASC])
                         +- Calc(select=[f])
-                           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -252,13 +252,13 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, d1)], select=[a, b, c, d, e,
 :- SortMergeJoin(joinType=[InnerJoin], where=[=(a, a1)], select=[a, b, c, d, e, f, a1, b1, c1], leftSorted=[true])
 :  :- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :  :  :- Exchange(distribution=[hash[a]])
-:  :  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  +- Exchange(distribution=[hash[d]])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[hash[a1]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 +- Exchange(distribution=[hash[d1]])
-   +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
 ]]>
     </Resource>
   </TestCase>
@@ -289,13 +289,13 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a1, d1)], select=[a, b, c, d, e
 :- SortMergeJoin(joinType=[InnerJoin], where=[=(d, a1)], select=[a, b, c, d, e, f, a1, b1, c1], leftSorted=[true])
 :  :- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :  :  :- Exchange(distribution=[hash[a]])
-:  :  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  +- Exchange(distribution=[hash[d]])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[hash[a1]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 +- Exchange(distribution=[hash[d1]])
-   +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
 ]]>
     </Resource>
   </TestCase>
@@ -381,7 +381,7 @@ Calc(select=[id, tb2_ids, tb3_ids, name, tb2_names, tb3_names, name0, name1])
    :     :        :- Calc(select=[id, key, tb2_ids, tb3_ids, name, tb2_names])
    :     :        :  +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(id, tb1_id)], select=[id, key, tb2_ids, tb3_ids, name, tb1_id, tb2_names], rightSorted=[true])
    :     :        :     :- Exchange(distribution=[hash[id]])
-   :     :        :     :  +- TableSourceScan(table=[[default_catalog, default_database, tb1, source: [TestTableSource(id, key, tb2_ids, tb3_ids, name)]]], fields=[id, key, tb2_ids, tb3_ids, name], reuse_id=[1])
+   :     :        :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, tb1, source: [TestTableSource(id, key, tb2_ids, tb3_ids, name)]]], fields=[id, key, tb2_ids, tb3_ids, name], reuse_id=[1])
    :     :        :     +- SortAggregate(isMerge=[true], groupBy=[tb1_id], select=[tb1_id, Final_LISTAGG(accDelimiter$0, concatAcc$1) AS tb2_names])
    :     :        :        +- Sort(orderBy=[tb1_id ASC])
    :     :        :           +- Exchange(distribution=[hash[tb1_id]])
@@ -394,7 +394,7 @@ Calc(select=[id, tb2_ids, tb3_ids, name, tb2_names, tb3_names, name0, name1])
    :     :        :                          :     +- Correlate(invocation=[split($cor0.tb2_ids)], correlate=[table(split($cor0.tb2_ids))], select=[id,key,tb2_ids,tb3_ids,name,f0], rowType=[RecordType(VARCHAR(2147483647) id, VARCHAR(2147483647) key, VARCHAR(2147483647) tb2_ids, VARCHAR(2147483647) tb3_ids, VARCHAR(2147483647) name, VARCHAR(2147483647) f0)], joinType=[INNER])
    :     :        :                          :        +- Reused(reference_id=[1])
    :     :        :                          +- Exchange(distribution=[hash[id]])
-   :     :        :                             +- TableSourceScan(table=[[default_catalog, default_database, tb2, source: [TestTableSource(id, name)]]], fields=[id, name])
+   :     :        :                             +- LegacyTableSourceScan(table=[[default_catalog, default_database, tb2, source: [TestTableSource(id, name)]]], fields=[id, name])
    :     :        +- SortAggregate(isMerge=[true], groupBy=[tb1_id], select=[tb1_id, Final_LISTAGG(accDelimiter$0, concatAcc$1) AS tb3_names])
    :     :           +- Sort(orderBy=[tb1_id ASC])
    :     :              +- Exchange(distribution=[hash[tb1_id]])
@@ -407,11 +407,11 @@ Calc(select=[id, tb2_ids, tb3_ids, name, tb2_names, tb3_names, name0, name1])
    :     :                             :     +- Correlate(invocation=[split($cor1.tb3_ids)], correlate=[table(split($cor1.tb3_ids))], select=[id,key,tb2_ids,tb3_ids,name,f0], rowType=[RecordType(VARCHAR(2147483647) id, VARCHAR(2147483647) key, VARCHAR(2147483647) tb2_ids, VARCHAR(2147483647) tb3_ids, VARCHAR(2147483647) name, VARCHAR(2147483647) f0)], joinType=[INNER])
    :     :                             :        +- Reused(reference_id=[1])
    :     :                             +- Exchange(distribution=[hash[id]])
-   :     :                                +- TableSourceScan(table=[[default_catalog, default_database, tb3, source: [TestTableSource(id, name)]]], fields=[id, name])
+   :     :                                +- LegacyTableSourceScan(table=[[default_catalog, default_database, tb3, source: [TestTableSource(id, name)]]], fields=[id, name])
    :     +- Exchange(distribution=[hash[id]])
-   :        +- TableSourceScan(table=[[default_catalog, default_database, tb4, source: [TestTableSource(id, name)]]], fields=[id, name])
+   :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, tb4, source: [TestTableSource(id, name)]]], fields=[id, name])
    +- Exchange(distribution=[hash[id]])
-      +- TableSourceScan(table=[[default_catalog, default_database, tb5, source: [TestTableSource(id, name)]]], fields=[id, name])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, tb5, source: [TestTableSource(id, name)]]], fields=[id, name])
 ]]>
     </Resource>
   </TestCase>
@@ -442,13 +442,13 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[AND(=(a, d1), =(b, e1))], select=
 :- SortMergeJoin(joinType=[InnerJoin], where=[AND(=(a, a1), =(b, b1))], select=[a, b, c, d, e, f, a1, b1, c1], leftSorted=[true])
 :  :- SortMergeJoin(joinType=[InnerJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c, d, e, f])
 :  :  :- Exchange(distribution=[hash[a, b]])
-:  :  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  +- Exchange(distribution=[hash[d, e]])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[hash[a1, b1]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 +- Exchange(distribution=[hash[d1, e1]])
-   +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
 ]]>
     </Resource>
   </TestCase>
@@ -479,13 +479,13 @@ SortMergeJoin(joinType=[LeftOuterJoin], where=[AND(=(a1, d1), =(b1, e1))], selec
 :- SortMergeJoin(joinType=[InnerJoin], where=[AND(=(d, a1), =(e, b1))], select=[a, b, c, d, e, f, a1, b1, c1], leftSorted=[true])
 :  :- SortMergeJoin(joinType=[InnerJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c, d, e, f])
 :  :  :- Exchange(distribution=[hash[a, b]])
-:  :  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  +- Exchange(distribution=[hash[d, e]])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[hash[a1, b1]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 +- Exchange(distribution=[hash[d1, e1]])
-   +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(d1, e1, f1)]]], fields=[d1, e1, f1])
 ]]>
     </Resource>
   </TestCase>
@@ -518,7 +518,7 @@ Calc(select=[sum_b, /(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT), w0$o0) AS av
             +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0])
                +- Sort(orderBy=[a ASC])
                   +- Calc(select=[a, b])
-                     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -544,7 +544,7 @@ Sort(orderBy=[a ASC])
    +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b, Final_COUNT(count$0) AS cnt])
       +- Exchange(distribution=[hash[a, b]])
          +- LocalHashAggregate(groupBy=[a, b], select=[a, b, Partial_COUNT(c) AS count$0])
-            +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -576,7 +576,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1])
          +- Exchange(distribution=[hash[a]])
             +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orderBy=[b ASC], global=[false], select=[a, b, c])
                +- Sort(orderBy=[a ASC, b ASC])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -607,7 +607,7 @@ SortAggregate(isMerge=[false], groupBy=[a, b], select=[a, b, MAX(c) AS EXPR$2])
       +- Exchange(distribution=[hash[a]])
          +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orderBy=[b ASC], global=[false], select=[a, b, c])
             +- Sort(orderBy=[a ASC, b ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -636,7 +636,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde
    +- Exchange(distribution=[hash[a]])
       +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orderBy=[b ASC], global=[false], select=[a, b, c])
          +- Sort(orderBy=[a ASC, b ASC])
-            +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -669,7 +669,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde
          +- LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(c) AS count$0])
             +- Sort(orderBy=[a ASC])
                +- Calc(select=[a, c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -701,7 +701,7 @@ SortAggregate(isMerge=[false], select=[COUNT(a) AS EXPR$0, SUM(b) AS EXPR$1])
          +- Exchange(distribution=[single])
             +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[], orderBy=[b ASC], global=[false], select=[a, b, c])
                +- Sort(orderBy=[b ASC])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.xml
index 8d971ce..edd1d4d 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.xml
@@ -41,7 +41,7 @@ Calc(select=[EXPR$0])
       +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_COUNT(count$0) AS cnt])
          +- Exchange(distribution=[hash[c]])
             +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_COUNT(b) AS count$0])
-               +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -75,14 +75,14 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b
 :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left])
 :  :- Exchange(distribution=[broadcast])
 :  :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
       :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -112,14 +112,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f1=[$3], f=[$4], cnt=[$5])
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f1, f, cnt], build=[right])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[f0 AS f1, f, cnt])
    +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
       +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
          +- Exchange(distribution=[hash[f]])
             +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                +- Calc(select=[f])
-                  +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -149,14 +149,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f=[$3], f1=[$4])
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[=(c, f)], select=[a, b, c, f, f1], build=[right])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[f, f0 AS f1])
    +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
       +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
          +- Exchange(distribution=[hash[f]])
             +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                +- Calc(select=[f], where=[LIKE(f, _UTF-16LE'%llo%')])
-                  +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -186,7 +186,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], f1=[$3])
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[=(c, f1)], select=[a, b, c, f1], build=[right])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f1]])
    +- Calc(select=[f0 AS f1])
       +- Correlate(invocation=[split($cor0.f)], correlate=[table(split($cor0.f))], select=[f,cnt,f0], rowType=[RecordType(VARCHAR(2147483647) f, BIGINT cnt, VARCHAR(2147483647) f0)], joinType=[INNER])
@@ -194,7 +194,7 @@ HashJoin(joinType=[InnerJoin], where=[=(c, f1)], select=[a, b, c, f1], build=[ri
             +- Exchange(distribution=[hash[f]])
                +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(f) AS count$0])
                   +- Calc(select=[f])
-                     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -224,10 +224,10 @@ Calc(select=[EXPR$0])
       +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
          :- Exchange(distribution=[hash[a]])
          :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
          +- Exchange(distribution=[hash[d]])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -256,10 +256,10 @@ Calc(select=[EXPR$0])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
          +- Calc(select=[d])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -289,10 +289,10 @@ Calc(select=[EXPR$0])
       +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
          :- Exchange(distribution=[hash[a]])
          :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
          +- Exchange(distribution=[hash[d]])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -326,15 +326,15 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b
 :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :  :- Exchange(distribution=[hash[a]])
 :  :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
    :- Exchange(distribution=[hash[a]])
    :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
-      +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -365,11 +365,11 @@ HashJoin(joinType=[InnerJoin], where=[AND(=(a, a0), =(c, c0))], select=[a, c, EX
 :- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS EXPR$2])
 :  +- Exchange(distribution=[hash[a, c]])
 :     +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0])
-:        +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS EXPR$2])
    +- Exchange(distribution=[hash[a, c]])
       +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0])
-         +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -405,17 +405,17 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b
 :- Exchange(distribution=[hash[a]])
 :  +- HashJoin(joinType=[FullOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
 :     :- Exchange(distribution=[hash[a]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[d]])
    +- HashJoin(joinType=[FullOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
       :- Exchange(distribution=[hash[a]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
          +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -450,17 +450,17 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7],
 HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b0, c0, d0, e0, f0], build=[right])
 :- HashJoin(joinType=[LeftOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
 :  :- Exchange(distribution=[hash[a]])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
 :     +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[d]])
    +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
       :- Exchange(distribution=[hash[a]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
          +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -496,16 +496,16 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b
 :- Exchange(distribution=[hash[a]])
 :  +- HashJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
 :     :- Exchange(distribution=[hash[a]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- HashJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[right])
    :- Exchange(distribution=[hash[a]])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-         +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -531,12 +531,12 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], cnt=[$4])
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[AND(=(a, d), =(b, cnt))], select=[a, b, c, d, cnt], build=[right])
 :- Exchange(distribution=[hash[a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS cnt])
    +- Exchange(distribution=[hash[d]])
       +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(f) AS count$0])
          +- Calc(select=[d, f])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -571,7 +571,7 @@ Calc(select=[sum_b, /(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT), w0$o0) AS av
                +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS sum_b])
                   +- Exchange(distribution=[hash[a, c]])
                      +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0])
-                        +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -606,13 +606,13 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :  +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :     :- Exchange(distribution=[broadcast])
 :     :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -645,7 +645,7 @@ Calc(select=[sum_b, /(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT), w0$o0) AS av
          +- Exchange(distribution=[hash[c]])
             +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0])
                +- Calc(select=[c, b])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -678,7 +678,7 @@ Calc(select=[sum_b, /(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT), w0$o0) AS av
          +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS sum_b])
             +- Exchange(distribution=[hash[c]])
                +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -710,7 +710,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde
       +- Exchange(distribution=[hash[a]])
          +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0])
             +- Calc(select=[a, b])
-               +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -742,7 +742,7 @@ HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1])
          +- Exchange(distribution=[hash[a]])
             +- Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a, c], orderBy=[b ASC], global=[false], select=[a, b, c])
                +- Sort(orderBy=[a ASC, c ASC, b ASC])
-                  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -772,7 +772,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a, c], o
    +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b, Final_COUNT(count$1) AS c])
       +- Exchange(distribution=[hash[a]])
          +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
-            +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -802,7 +802,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a, c], o
    +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b, Final_COUNT(count$1) AS c])
       +- Exchange(distribution=[hash[a]])
          +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
-            +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -834,7 +834,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[], order
       +- Exchange(distribution=[single])
          +- LocalHashAggregate(select=[Partial_COUNT(a) AS count$0, Partial_SUM(b) AS sum$1])
             +- Calc(select=[a, b])
-               +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -866,7 +866,7 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde
       +- Exchange(distribution=[hash])
          +- LocalHashAggregate(select=[Partial_COUNT(a) AS count$0, Partial_SUM(b) AS sum$1])
             +- Calc(select=[a, b])
-               +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -897,10 +897,10 @@ Calc(select=[EXPR$0])
          +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
             :- Exchange(distribution=[hash[a]])
             :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
             +- Exchange(distribution=[hash[d]])
                +- Calc(select=[d])
-                  +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -930,10 +930,10 @@ Calc(select=[EXPR$0])
       +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
          :- Exchange(distribution=[hash[a]])
          :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
          +- Exchange(distribution=[hash[d]])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -964,10 +964,10 @@ Calc(select=[EXPR$0])
          +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d], build=[left])
             :- Exchange(distribution=[hash[a]])
             :  +- Calc(select=[a, b], where=[LIKE(c, _UTF-16LE'He%')])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
             +- Exchange(distribution=[hash[d]])
                +- Calc(select=[d])
-                  +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1001,15 +1001,15 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :  :- Exchange(distribution=[hash[a]])
 :  :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
    :- Exchange(distribution=[hash[a]])
    :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
-      +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1045,17 +1045,17 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :- Exchange(distribution=[hash[a]])
 :  +- SortMergeJoin(joinType=[FullOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :     :- Exchange(distribution=[hash[a]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[d]])
    +- SortMergeJoin(joinType=[FullOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
       :- Exchange(distribution=[hash[a]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
          +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1090,17 +1090,17 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7],
 SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b0, c0, d0, e0, f0], leftSorted=[true])
 :- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :  :- Exchange(distribution=[hash[a]])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
 :     +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[d]])
    +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
       :- Exchange(distribution=[hash[a]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
          +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-            +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1136,16 +1136,16 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :- Exchange(distribution=[hash[a]])
 :  +- SortMergeJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
 :     :- Exchange(distribution=[hash[a]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- SortMergeJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
    :- Exchange(distribution=[hash[a]])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d, CAST(2:BIGINT) AS e, f], where=[=(e, 2:BIGINT)])
-         +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1176,7 +1176,7 @@ Calc(select=[EXPR$0])
    :     +- Exchange(distribution=[hash[c]])
    :        +- LocalHashAggregate(groupBy=[c], select=[c, Partial_AVG(b) AS (sum$0, count$1), Partial_SUM(b) AS sum$2])
    :           +- Calc(select=[c, b])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0])
          +- Calc(select=[*($f0, 0.1:DECIMAL(2, 1)) AS EXPR$0])
@@ -1184,7 +1184,7 @@ Calc(select=[EXPR$0])
                +- Exchange(distribution=[single])
                   +- LocalSortAggregate(select=[Partial_SUM(b) AS sum$0])
                      +- Calc(select=[b])
-                        +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1238,26 +1238,26 @@ Calc(select=[c, cnt, c0, cnt0])
    :     :     +- Exchange(distribution=[hash[c]])
    :     :        +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(a) AS count$0])
    :     :           +- Calc(select=[c, a], where=[>(b, 10)])
-   :     :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Calc(select=[cnt, f], where=[<(cnt, 10)])
    :        +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
    :           +- Exchange(distribution=[hash[f]])
    :              +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(d) AS count$0])
    :                 +- Calc(select=[f, d], where=[<(e, 100)])
-   :                    +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Union(all=[true], union=[cnt, c])
       :- Calc(select=[cnt, c])
       :  +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_COUNT(count$0) AS cnt])
       :     +- Exchange(distribution=[hash[c]])
       :        +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(a) AS count$0])
       :           +- Calc(select=[c, a], where=[>(b, 10)])
-      :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Calc(select=[cnt, f])
          +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt])
             +- Exchange(distribution=[hash[f]])
                +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(d) AS count$0])
                   +- Calc(select=[f, d], where=[<(e, 100)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml
index 8b3ae12..3b188b7 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml
@@ -35,10 +35,10 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], build=[left])
    :- Exchange(distribution=[hash[c]])
    :  +- Calc(select=[c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[f]])
       +- Calc(select=[f])
-         +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -65,9 +65,9 @@ Calc(select=[c0 AS c])
             +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(vcol_left_marker) AS count$0, Partial_COUNT(vcol_right_marker) AS count$1])
                +- Union(all=[true], union=[c, vcol_left_marker, vcol_right_marker])
                   :- Calc(select=[c, true AS vcol_left_marker, null:BOOLEAN AS vcol_right_marker])
-                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
                   +- Calc(select=[f, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker])
-                     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -132,10 +132,10 @@ Calc(select=[a])
    +- HashJoin(joinType=[LeftSemiJoin], where=[AND(IS NOT DISTINCT FROM(a, d), IS NOT DISTINCT FROM(b, e))], select=[a, b], build=[left])
       :- Exchange(distribution=[hash[a, b]])
       :  +- Calc(select=[a, b])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d, e]])
          +- Calc(select=[d, e])
-            +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -158,10 +158,10 @@ HashAggregate(isMerge=[false], groupBy=[c], select=[c])
 +- HashJoin(joinType=[LeftAntiJoin], where=[IS NOT DISTINCT FROM(c, f)], select=[c], build=[left])
    :- Exchange(distribution=[hash[c]])
    :  +- Calc(select=[c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[f]])
       +- Calc(select=[f])
-         +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -188,9 +188,9 @@ Calc(select=[c0 AS c])
             +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(vcol_marker) AS sum$0])
                +- Union(all=[true], union=[c, vcol_marker])
                   :- Calc(select=[c, 1:BIGINT AS vcol_marker])
-                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
                   +- Calc(select=[f, -1:BIGINT AS vcol_marker])
-                     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -234,7 +234,7 @@ HashAggregate(isMerge=[true], groupBy=[c], select=[c])
 +- Exchange(distribution=[hash[c]])
    +- LocalHashAggregate(groupBy=[c], select=[c])
       +- Calc(select=[c])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -256,7 +256,7 @@ LogicalMinus(all=[false])
 HashAggregate(isMerge=[false], groupBy=[a, b, c], select=[a, b, c])
 +- HashJoin(joinType=[LeftAntiJoin], where=[AND(IS NOT DISTINCT FROM(a, a0), IS NOT DISTINCT FROM(b, b0), IS NOT DISTINCT FROM(c, c0))], select=[a, b, c], build=[left])
    :- Exchange(distribution=[hash[a, b, c]], shuffle_mode=[BATCH], reuse_id=[1])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -278,7 +278,7 @@ LogicalUnion(all=[true])
       <![CDATA[
 Union(all=[true], union=[a])
 :- Calc(select=[a])
-:  +- TableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
 +- Calc(select=[CASE(>(c, 0), b, null:RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" _2)) AS EXPR$0])
    +- Reused(reference_id=[1])
 ]]>
@@ -301,7 +301,7 @@ LogicalUnion(all=[true])
       <![CDATA[
 Union(all=[true], union=[a])
 :- Calc(select=[a])
-:  +- TableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b)]]], fields=[a, b], reuse_id=[1])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a, b)]]], fields=[a, b], reuse_id=[1])
 +- Calc(select=[b])
    +- Reused(reference_id=[1])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml
index 74112af..4c24756 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.xml
@@ -32,7 +32,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs
 SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true])
 +- Exchange(distribution=[single])
    +- SortLimit(orderBy=[a DESC, b ASC], offset=[0], fetch=[11], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -68,7 +68,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], offset=[10])
       <![CDATA[
 SortLimit(orderBy=[a DESC], offset=[10], fetch=[unlimited], global=[true])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -105,7 +105,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[5])
 SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true])
 +- Exchange(distribution=[single])
    +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -124,7 +124,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], offset=[10])
       <![CDATA[
 SortLimit(orderBy=[a DESC], offset=[10], fetch=[unlimited], global=[true])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -161,7 +161,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last], fetch=[5])
 SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[true])
 +- Exchange(distribution=[single])
    +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[5], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -181,7 +181,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs
 SortLimit(orderBy=[a DESC, b ASC], offset=[1], fetch=[10], global=[true])
 +- Exchange(distribution=[single])
    +- SortLimit(orderBy=[a DESC, b ASC], offset=[0], fetch=[11], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml
index d08cb76..74e6374 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SortTest.xml
@@ -31,7 +31,7 @@ LogicalSort(sort0=[$0], sort1=[$1], dir0=[DESC-nulls-last], dir1=[ASC-nulls-firs
       <![CDATA[
 Sort(orderBy=[a DESC, b ASC])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -50,7 +50,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last])
       <![CDATA[
 Sort(orderBy=[a DESC])
 +- Exchange(distribution=[range[a DESC]])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -69,7 +69,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last])
       <![CDATA[
 Sort(orderBy=[a DESC])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -88,7 +88,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last])
       <![CDATA[
 Sort(orderBy=[a DESC])
 +- Exchange(distribution=[range[a DESC]])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -108,7 +108,7 @@ LogicalSort(sort0=[$0], dir0=[DESC-nulls-last])
 SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[true])
 +- Exchange(distribution=[single])
    +- SortLimit(orderBy=[a DESC], offset=[0], fetch=[200], global=[false])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
index 157aadbb..63db042 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
@@ -45,7 +45,7 @@ Calc(select=[a])
    :     +- Limit(offset=[0], fetch=[10], global=[true])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Reused(reference_id=[1])
 ]]>
@@ -80,7 +80,7 @@ Calc(select=[a])
    :     +- Limit(offset=[0], fetch=[10], global=[true])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[any], shuffle_mode=[BATCH])
       +- Reused(reference_id=[1])
 ]]>
@@ -117,16 +117,16 @@ HashJoin(joinType=[InnerJoin], where=[=(b, e0)], select=[a, b, c, d, e, f, a0, b
 :  +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b, c], where=[<(a, 10)])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
-:        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[e]])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b, c], where=[>(a, 5)])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -161,10 +161,10 @@ HashJoin(joinType=[InnerJoin], where=[=(b, e0)], select=[a, b, d, e, a0, b0, d0,
 :  +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d, e], build=[left])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[<(a, 10)])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
 :     +- Exchange(distribution=[hash[d]], reuse_id=[2])
 :        +- Calc(select=[d, e])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[e]])
    +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, d, e], build=[left])
       :- Exchange(distribution=[hash[a]])
@@ -253,10 +253,10 @@ Calc(select=[c, e, avg_b, sum_b, sum_b0 AS psum, sum_b1 AS nsum, avg_b0 AS avg_b
    :     :                       +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e], build=[right])
    :     :                          :- Exchange(distribution=[hash[a]])
    :     :                          :  +- Calc(select=[a, b, c], where=[IS NOT NULL(c)])
-   :     :                          :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :                          :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :                          +- Exchange(distribution=[hash[d]])
    :     :                             +- Calc(select=[d, e], where=[>(e, 10)])
-   :     :                                +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :     :                                +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    :     +- Exchange(distribution=[hash[c, e, $f5]], shuffle_mode=[BATCH])
    :        +- Calc(select=[sum_b, /(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT), w0$o0) AS avg_b, c, e, +(w1$o0, 1) AS $f5])
    :           +- Reused(reference_id=[1])
@@ -296,7 +296,7 @@ Calc(select=[c, a, b, c0, a1, b0])
    :     +- SortAggregate(isMerge=[false], groupBy=[c], select=[c, MyFirst(a) AS a, MyLast(b) AS b], reuse_id=[1])
    :        +- Sort(orderBy=[c ASC])
    :           +- Exchange(distribution=[hash[c]])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[c, a, b], where=[>(a, 1)])
          +- Reused(reference_id=[1])
@@ -338,10 +338,10 @@ Union(all=[true], union=[a, c, e])
 :  +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, c, d, e], build=[left])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, c], where=[LIKE(c, _UTF-16LE'test%')])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d, e])
-:           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -369,7 +369,7 @@ LogicalUnion(all=[true])
       <![CDATA[
 Union(all=[true], union=[a, EXPR$1])
 :- Calc(select=[a, random_udf() AS EXPR$1], where=[>(a, 10)], reuse_id=[1])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -397,7 +397,7 @@ LogicalUnion(all=[true])
       <![CDATA[
 Union(all=[true], union=[a])
 :- Calc(select=[a], where=[>(b, random_udf(a))], reuse_id=[1])
-:  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -429,10 +429,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], v=[$3], a0=[$4], b0=[$5], c0=[$6], v0=[$7
 HashJoin(joinType=[InnerJoin], where=[=(f0, f00)], select=[a, b, c, f0, a0, b0, c0, f00], build=[right])
 :- Exchange(distribution=[hash[f0]])
 :  +- Correlate(invocation=[str_split($cor0.c, _UTF-16LE'-')], correlate=[table(str_split($cor0.c,_UTF-16LE'-'))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)], joinType=[INNER])
-:     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f0]])
    +- Correlate(invocation=[str_split($cor1.c, _UTF-16LE'-')], correlate=[table(str_split($cor1.c,_UTF-16LE'-'))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)], joinType=[INNER])
-      +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -463,10 +463,10 @@ LogicalProject(a=[$0], b=[$1], c=[$2], s=[$3], a0=[$4], b0=[$5], c0=[$6], s0=[$7
 HashJoin(joinType=[InnerJoin], where=[=(c, f00)], select=[a, b, c, f0, a0, b0, c0, f00], build=[right])
 :- Correlate(invocation=[TableFun($cor0.c)], correlate=[table(TableFun($cor0.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)], joinType=[INNER])
 :  +- Exchange(distribution=[hash[c]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f0]])
    +- Correlate(invocation=[TableFun($cor1.c)], correlate=[table(TableFun($cor1.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)], joinType=[INNER])
-      +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -499,7 +499,7 @@ Calc(select=[a, b, a1 AS a0])
    :     +- Limit(offset=[0], fetch=[10], global=[true], reuse_id=[1])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[a, b])
          +- Reused(reference_id=[1])
@@ -569,15 +569,15 @@ Union(all=[true], union=[a, b, c, d, e, f])
 :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :  :- Exchange(distribution=[hash[a]], reuse_id=[1])
 :  :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'test%')])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
 :     +- Calc(select=[d, e, f], where=[>(e, 10)])
-:        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
    :- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d, e, f], where=[<>(f, _UTF-16LE'':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
-         +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -610,7 +610,7 @@ Calc(select=[c, a, b, c0, a1, b0])
    :     +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b], reuse_id=[1])
    :        +- Exchange(distribution=[hash[c]])
    :           +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[c, a, b], where=[>(a, 1)])
          +- Reused(reference_id=[1])
@@ -648,9 +648,9 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b
 :  +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left], reuse_id=[1])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
-:        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -689,9 +689,9 @@ Calc(select=[a, b, c, d, e, f, a1, b0, c0, d0, e0, f0])
    :     +- NestedLoopJoin(joinType=[InnerJoin], where=[OR(=(ABS(a), ABS(d)), =(c, f))], select=[a, b, c, d, e, f], build=[left], reuse_id=[1])
    :        :- Exchange(distribution=[broadcast])
    :        :  +- Calc(select=[a, b, c], where=[>(b, 1)])
-   :        :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :        :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        +- Calc(select=[d, e, f], where=[<(e, 2)])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[hash[b]])
       +- Reused(reference_id=[1])
 ]]>
@@ -731,9 +731,9 @@ Calc(select=[a, b, c, d, e, f, a1, b0, c0, d0, e0, f0])
    :     +- NestedLoopJoin(joinType=[InnerJoin], where=[OR(=(random_udf(a), random_udf(d)), =(c, f))], select=[a, b, c, d, e, f], build=[left], reuse_id=[1])
    :        :- Exchange(distribution=[broadcast])
    :        :  +- Calc(select=[a, b, c], where=[>(b, 1)])
-   :        :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :        :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        +- Calc(select=[d, e, f], where=[<(e, 2)])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[hash[b]])
       +- Reused(reference_id=[1])
 ]]>
@@ -768,7 +768,7 @@ Calc(select=[a, b, a1 AS a0])
    :     +- Limit(offset=[0], fetch=[10], global=[true], reuse_id=[1])
    :        +- Exchange(distribution=[single])
    :           +- Limit(offset=[0], fetch=[10], global=[false])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[a, b])
          +- Reused(reference_id=[1])
@@ -806,8 +806,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :  +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left], reuse_id=[1])
 :     :- Exchange(distribution=[broadcast])
 :     :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[any], shuffle_mode=[BATCH])
    +- Reused(reference_id=[1])
 ]]>
@@ -839,7 +839,7 @@ HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, $2, a0, b0, $20],
 :     +- OverAggregate(partitionBy=[c], orderBy=[c DESC], window#0=[MyFirst(c) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0], reuse_id=[1])
 :        +- Sort(orderBy=[c DESC])
 :           +- Exchange(distribution=[hash[c]])
-:              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[a]])
    +- Calc(select=[a, b, w0$o0 AS $2], where=[>(b, 10)])
       +- Reused(reference_id=[1])
@@ -874,7 +874,7 @@ HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[c, a, b, c0, a0, b0], b
 :     +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b], reuse_id=[1])
 :        +- Exchange(distribution=[hash[c]])
 :           +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
-:              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[a]])
    +- Calc(select=[c, a, b], where=[<(b, 10)])
       +- Reused(reference_id=[1])
@@ -912,7 +912,7 @@ Calc(select=[c, a, b, c0, a1, b0])
    :           +- Exchange(distribution=[hash[c]])
    :              +- LocalSortAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
    :                 +- Sort(orderBy=[c ASC])
-   :                    +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[c, a, b], where=[>(a, 1)])
          +- Reused(reference_id=[1])
@@ -945,7 +945,7 @@ HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, $2, a0, b0, $20],
 :     +- OverAggregate(orderBy=[c DESC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0], reuse_id=[1])
 :        +- Sort(orderBy=[c DESC])
 :           +- Exchange(distribution=[single])
-:              +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[a]])
    +- Calc(select=[a, b, w0$o0 AS $2], where=[>(b, 10)])
       +- Reused(reference_id=[1])
@@ -985,7 +985,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[c, a, b, c0, a0,
 :              +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS a, Final_SUM(sum$1) AS b])
 :                 +- Exchange(distribution=[hash[c]])
 :                    +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
-:                       +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[any], shuffle_mode=[BATCH])
    +- Calc(select=[c, a, b], where=[<(b, 10)])
       +- Reused(reference_id=[1])
@@ -1027,9 +1027,9 @@ Calc(select=[a, c, c0])
    :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH], reuse_id=[1])
    :  +- Union(all=[true], union=[a, c])
    :     :- Calc(select=[a, c], where=[>(b, 10)])
-   :     :  +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Calc(select=[d, f], where=[<(e, 100)])
-   :        +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -1064,9 +1064,9 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f,
 :- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], reuse_id=[1])
 :  :- Exchange(distribution=[hash[a]])
 :  :  +- Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'He%')])
-:  :     +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[hash[d]])
-:     +- TableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -1102,7 +1102,7 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[IS NOT DISTINCT FROM(random, rand
 :        :        +- Exchange(distribution=[single])
 :        :           +- SortLimit(orderBy=[EXPR$1 ASC], offset=[0], fetch=[1], global=[false])
 :        :              +- Calc(select=[a AS random, RAND() AS EXPR$1])
-:        :                 +- TableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:        :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :        +- Exchange(distribution=[broadcast], reuse_id=[2])
 :           +- Reused(reference_id=[1])
 +- Reused(reference_id=[2])
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml
index ad8a100..0d31a72 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml
@@ -28,7 +28,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -46,7 +46,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[>(price, 10)])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -64,7 +64,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), >(price, 10))])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -82,7 +82,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[OR(>(amount, 2), <(amount, 10))])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -99,7 +99,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[and(greaterThan(amount, 2), lessThan(amount, 10))]]]], fields=[name, id, amount, price])
+LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[and(greaterThan(amount, 2), lessThan(amount, 10))]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -116,7 +116,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
+LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -134,7 +134,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[>(price, 10)])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -155,7 +155,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[AND(<(id, 100), >(CAST(amount), 10))])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -173,7 +173,7 @@ LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[name, id, amount, price], where=[<(Func1$(amount), 32)])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable, source: [filterPushedDown=[true], filter=[greaterThan(amount, 2)]]]], fields=[name, id, amount, price])
 ]]>
     </Resource>
   </TestCase>
@@ -197,7 +197,7 @@ LogicalProject(id=[$0], nestedName=[$1.nested1.name], nestedValue=[$2.value], ne
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[id, deepNested.nested1.name AS nestedName, nested.value AS nestedValue, deepNested.nested2.flag AS nestedFlag, deepNested.nested2.num AS nestedNum])
-+- TableSourceScan(table=[[default_catalog, default_database, T, source: [TestSource(read nested fields: id.*, deepNested.nested2.num, deepNested.nested2.flag, deepNested.nested1.name, nested.value)]]], fields=[id, deepNested, nested])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, T, source: [TestSource(read nested fields: id.*, deepNested.nested2.num, deepNested.nested2.flag, deepNested.nested1.name, nested.value)]]], fields=[id, deepNested, nested])
 ]]>
     </Resource>
   </TestCase>
@@ -215,7 +215,7 @@ LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[id, name, CAST(_UTF-16LE'A':VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS part1, part2], where=[>(id, 2)])
-+- TableSourceScan(table=[[default_catalog, default_database, PartitionableTable, source: [partitions={part1=A, part2=2}]]], fields=[id, name, part1, part2])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, PartitionableTable, source: [partitions={part1=A, part2=2}]]], fields=[id, name, part1, part2])
 ]]>
     </Resource>
   </TestCase>
@@ -233,7 +233,7 @@ LogicalProject(id=[$0], name=[$1], part1=[$2], part2=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[id, name, part1, part2], where=[>(id, 2)])
-+- TableSourceScan(table=[[default_catalog, default_database, PartitionableTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]], fields=[id, name, part1, part2])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, PartitionableTable, source: [partitions={part1=A, part2=1}, {part1=C, part2=1}]]], fields=[id, name, part1, part2])
 ]]>
     </Resource>
   </TestCase>
@@ -253,7 +253,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
 SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0])
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
-      +- TableSourceScan(table=[[default_catalog, default_database, ProjectableTable, source: [TestSource(physical fields: )]]], fields=[])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, ProjectableTable, source: [TestSource(physical fields: )]]], fields=[])
 ]]>
     </Resource>
   </TestCase>
@@ -276,7 +276,7 @@ LogicalProject(id=[$0])
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[id])
-+- TableSourceScan(table=[[default_catalog, default_database, FilterableTable1, source: [filterPushedDown=[true], filter=[and(and(greaterThan(tv, 14:25:02), greaterThan(dv, 2017-02-03)), greaterThan(tsv, 2017-02-03T14:25:02))]]]], fields=[id, dv, tv, tsv])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, FilterableTable1, source: [filterPushedDown=[true], filter=[and(and(greaterThan(tv, 14:25:02), greaterThan(dv, 2017-02-03)), greaterThan(tsv, 2017-02-03T14:25:02))]]]], fields=[id, dv, tv, tsv])
 ]]>
     </Resource>
   </TestCase>
@@ -292,7 +292,7 @@ LogicalProject(a=[$0], c=[$2])
     </Resource>
     <Resource name="planAfter">
       <![CDATA[
-TableSourceScan(table=[[default_catalog, default_database, ProjectableTable, source: [TestSource(physical fields: a, c)]]], fields=[a, c])
+LegacyTableSourceScan(table=[[default_catalog, default_database, ProjectableTable, source: [TestSource(physical fields: a, c)]]], fields=[a, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml
index 5b02e4e..5af81bf 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnionTest.xml
@@ -47,11 +47,11 @@ LogicalProject(a=[$0], c=[$1])
 Union(all=[true], union=[a, c])
 :- Union(all=[true], union=[a, c])
 :  :- Calc(select=[a, c], where=[>(a, 2)])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Calc(select=[a, c], where=[>(a, 2)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[a, c], where=[>(a, 2)])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -78,9 +78,9 @@ LogicalProject(a=[$0], b=[$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)
       <![CDATA[
 Union(all=[true], union=[a, b]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)]
 :- Calc(select=[a, CAST(b) AS b]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b)]
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]
 +- Calc(select=[a, CAST(0.0:DECIMAL(20, 1)) AS EXPR$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) EXPR$1)]
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
index 6da75ba..192da32 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.xml
@@ -35,7 +35,7 @@ LogicalProject(a=[$0], s=[$3])
       <![CDATA[
 Calc(select=[a, f0 AS s])
 +- Correlate(invocation=[explode($cor0.c)], correlate=[table(explode($cor0.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) ARRAY c, VARCHAR(2147483647) f0)], joinType=[INNER])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -97,7 +97,7 @@ LogicalProject(a=[$0], b=[$1], v=[$4])
       <![CDATA[
 Calc(select=[a, b, f1 AS v])
 +- Correlate(invocation=[explode($cor0.c)], correlate=[table(explode($cor0.c))], select=[a,b,c,f0,f1], rowType=[RecordType(INTEGER a, BIGINT b, (VARCHAR(2147483647), VARCHAR(2147483647)) MAP c, VARCHAR(2147483647) f0, VARCHAR(2147483647) f1)], joinType=[INNER])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -129,7 +129,7 @@ LogicalProject(a=[$0], b=[$1], x=[$2], y=[$3])
 Calc(select=[a, b, _1, _2], where=[>(_1, a)])
 +- Correlate(invocation=[explode($cor0.b)], correlate=[table(explode($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER])
    +- Calc(select=[a, b], where=[<(a, 3)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
 ]]>
     </Resource>
   </TestCase>
@@ -187,7 +187,7 @@ LogicalProject(a=[$0], s=[$3])
       <![CDATA[
 Calc(select=[a, f0 AS s])
 +- Correlate(invocation=[explode($cor0.c)], correlate=[table(explode($cor0.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, INTEGER ARRAY b, INTEGER ARRAY ARRAY c, INTEGER ARRAY f0)], joinType=[INNER])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -245,7 +245,7 @@ LogicalProject(a=[$0], b=[$1], s=[$3])
       <![CDATA[
 Calc(select=[a, b, f0 AS s])
 +- Correlate(invocation=[explode($cor0.b)], correlate=[table(explode($cor0.b))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, INTEGER ARRAY b, INTEGER ARRAY ARRAY c, INTEGER f0)], joinType=[INNER])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -268,7 +268,7 @@ LogicalProject(a=[$0], b=[$1], s=[$2], t=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Correlate(invocation=[explode($cor0.b)], correlate=[table(explode($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER], condition=[>($0, 13)])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
 ]]>
     </Resource>
   </TestCase>
@@ -290,7 +290,7 @@ LogicalProject(a=[$0], b=[$1], _1=[$2], _2=[$3])
     <Resource name="planAfter">
       <![CDATA[
 Correlate(invocation=[explode($cor0.b)], correlate=[table(explode($cor0.b))], select=[a,b,_1,_2], rowType=[RecordType(INTEGER a, RecordType:peek_no_expand(INTEGER _1, VARCHAR(2147483647) _2) ARRAY b, INTEGER _1, VARCHAR(2147483647) _2)], joinType=[INNER], condition=[>($0, 1)])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b)]]], fields=[a, b])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml
index 3dab6fa..2102836 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.xml
@@ -37,10 +37,10 @@ Calc(select=[a1, b1, a2, b2, EXPR$4])
       +- HashJoin(joinType=[FullOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
          :- Exchange(distribution=[hash[a1]])
          :  +- Calc(select=[a1, b1, c1])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
          +- Exchange(distribution=[hash[b2]])
             +- Calc(select=[a2, b2])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 ]]>
     </Resource>
   </TestCase>
@@ -66,10 +66,10 @@ Calc(select=[a1, b1, a3, b3, EXPR$4])
          +- HashJoin(joinType=[FullOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], build=[right])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1, b1, c1])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
             +- Exchange(distribution=[hash[a3]])
                +- Calc(select=[a3, b3])
-                  +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -93,10 +93,10 @@ HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, a2, b2], select=[a
 +- HashJoin(joinType=[InnerJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
    :- Exchange(distribution=[hash[a1]])
    :  +- Calc(select=[a1, b1, c1])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
    +- Exchange(distribution=[hash[b2]])
       +- Calc(select=[a2, b2])
-         +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 ]]>
     </Resource>
   </TestCase>
@@ -121,10 +121,10 @@ Calc(select=[a2, b2, a3, b3, EXPR$4, EXPR$5])
    +- Exchange(distribution=[hash[a3, b3]])
       +- LocalHashAggregate(groupBy=[a3, b3], auxGrouping=[a2, b2], select=[a3, b3, a2, b2, Partial_COUNT(c2) AS count$0, Partial_AVG(d3) AS (sum$1, count$2)])
          +- HashJoin(joinType=[InnerJoin], where=[=(b2, a3)], select=[a2, b2, c2, a3, b3, d3], isBroadcast=[true], build=[right])
-            :- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+            :- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
             +- Exchange(distribution=[broadcast])
                +- Calc(select=[a3, b3, d3])
-                  +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -152,16 +152,16 @@ Calc(select=[a1, b1, a2, b2, a3, b3, *($f4, $f2) AS EXPR$6])
    :  +- HashJoin(joinType=[InnerJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
    :     :- Exchange(distribution=[hash[a1]])
    :     :  +- Calc(select=[a1, b1, c1])
-   :     :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
    :     +- Exchange(distribution=[hash[b2]])
    :        +- Calc(select=[a2, b2])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    +- Exchange(distribution=[broadcast])
       +- HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count1$0) AS $f2])
          +- Exchange(distribution=[hash[a3, b3]])
             +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(*) AS count1$0])
                +- Calc(select=[a3, b3])
-                  +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -184,10 +184,10 @@ HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, a2, b2], select=[a
 +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
    :- Exchange(distribution=[hash[a1]])
    :  +- Calc(select=[a1, b1, c1])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
    +- Exchange(distribution=[hash[b2]])
       +- Calc(select=[a2, b2])
-         +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 ]]>
     </Resource>
   </TestCase>
@@ -212,10 +212,10 @@ Calc(select=[a1, b1, a3, b3, EXPR$4])
       +- LocalHashAggregate(groupBy=[a1, a3, b3], auxGrouping=[b1], select=[a1, a3, b3, b1, Partial_COUNT(c1) AS count$0])
          +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], isBroadcast=[true], build=[right])
             :- Calc(select=[a1, b1, c1])
-            :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
             +- Exchange(distribution=[broadcast])
                +- Calc(select=[a3, b3])
-                  +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -238,10 +238,10 @@ HashAggregate(isMerge=[false], groupBy=[a3, b3], auxGrouping=[a1, b1], select=[a
 +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a3)], select=[a3, b3, a1, b1, c1], build=[left])
    :- Exchange(distribution=[hash[a3]])
    :  +- Calc(select=[a3, b3])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
    +- Exchange(distribution=[hash[a1]])
       +- Calc(select=[a1, b1, c1])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -266,7 +266,7 @@ Calc(select=[a1, b1, c, EXPR$3])
             +- OverAggregate(partitionBy=[c1], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a1, b1, c1, d1, w0$o0])
                +- Sort(orderBy=[c1 ASC])
                   +- Exchange(distribution=[hash[c1]])
-                     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -290,10 +290,10 @@ Calc(select=[a1, b1, a2, b2, EXPR$4])
    +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, b2)], select=[a1, b1, c1, a2, b2], build=[right])
       :- Exchange(distribution=[hash[a1]])
       :  +- Calc(select=[a1, b1, c1])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
       +- Exchange(distribution=[hash[b2]])
          +- Calc(select=[a2, b2])
-            +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 ]]>
     </Resource>
   </TestCase>
@@ -317,10 +317,10 @@ Calc(select=[a1, b1, a3, b3, EXPR$4])
    +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3], build=[right])
       :- Exchange(distribution=[hash[a1]])
       :  +- Calc(select=[a1, b1, c1])
-      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
       +- Exchange(distribution=[hash[a3]])
          +- Calc(select=[a3, b3])
-            +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -345,9 +345,9 @@ HashAggregate(isMerge=[true], groupBy=[a3, b3, a1], auxGrouping=[b1], select=[a3
       +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a3)], select=[a3, b3, a1, b1, c1], isBroadcast=[true], build=[left])
          :- Exchange(distribution=[broadcast])
          :  +- Calc(select=[a3, b3])
-         :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+         :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
          +- Calc(select=[a1, b1, c1])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -367,7 +367,7 @@ LogicalProject(a4=[$0], b4=[$1], EXPR$2=[$3])
       <![CDATA[
 HashWindowAggregate(groupBy=[a4], auxGrouping=[b4], window=[TumblingGroupWindow('w$, d4, 900000)], select=[a4, b4 AS EXPR$2, COUNT(c4) AS EXPR$2])
 +- Exchange(distribution=[hash[a4]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
 ]]>
     </Resource>
   </TestCase>
@@ -387,7 +387,7 @@ LogicalProject(a4=[$0], c4=[$1], EXPR$2=[$3], EXPR$3=[$4])
       <![CDATA[
 HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow('w$, d4, 900000)], select=[a4, c4 AS EXPR$2, COUNT(b4) AS EXPR$2, AVG(b4) AS EXPR$3])
 +- Exchange(distribution=[hash[a4]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
 ]]>
     </Resource>
   </TestCase>
@@ -414,7 +414,7 @@ Calc(select=[a4, c4, s, EXPR$3])
             +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow('w$, d4, 900000)], properties=[w$start, w$end, w$rowtime], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
                +- Calc(select=[a4, c4, d4, b4, *(b4, b4) AS $f4])
                   +- Exchange(distribution=[hash[a4]])
-                     +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
 ]]>
     </Resource>
   </TestCase>
@@ -441,7 +441,7 @@ Calc(select=[a4, c4, e, EXPR$3])
             +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow('w$, d4, 900000)], properties=[w$start, w$end, w$rowtime], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
                +- Calc(select=[a4, c4, d4, b4, *(b4, b4) AS $f4])
                   +- Exchange(distribution=[hash[a4]])
-                     +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
 ]]>
     </Resource>
   </TestCase>
@@ -467,7 +467,7 @@ HashAggregate(isMerge=[true], groupBy=[a4, b4], auxGrouping=[c4], select=[a4, b4
          +- HashWindowAggregate(groupBy=[a4], auxGrouping=[c4], window=[TumblingGroupWindow('w$, d4, 900000)], select=[a4, c4 AS $f2, SUM($f4) AS $f2, SUM(b4) AS $f3, COUNT(b4) AS $f4])
             +- Calc(select=[a4, c4, d4, b4, *(b4, b4) AS $f4])
                +- Exchange(distribution=[hash[a4]])
-                  +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4, d4)]]], fields=[a4, b4, c4, d4])
 ]]>
     </Resource>
   </TestCase>
@@ -488,7 +488,7 @@ Calc(select=[a1, b1, c1, EXPR$3])
    +- Exchange(distribution=[hash[a1, b1, c1, $e]])
       +- LocalHashAggregate(groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Partial_COUNT(d1) AS count$0])
          +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, b1, c1, d1, 4 AS $e}, {null AS a1, b1, null AS c1, d1, 5 AS $e}, {null AS a1, null AS b1, c1, d1, 6 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -508,7 +508,7 @@ Calc(select=[a1, b1, c1, EXPR$3])
 +- HashAggregate(isMerge=[false], groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, COUNT(d1) AS EXPR$3])
    +- Exchange(distribution=[hash[a1, b1, c1, $e]])
       +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, c1, d1, 2 AS $e}])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -532,7 +532,7 @@ Calc(select=[a1, s])
       +- LocalHashAggregate(groupBy=[a1, c1, $e], select=[a1, c1, $e, Partial_SUM(b1) AS sum$0])
          +- Expand(projects=[a1, c1, b1, $e], projects=[{a1, c1, b1, 0 AS $e}, {a1, null AS c1, b1, 1 AS $e}, {null AS a1, null AS c1, b1, 3 AS $e}])
             +- Calc(select=[a1, c1, b1])
-               +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -553,7 +553,7 @@ Calc(select=[a1, b1, c1, EXPR$3])
 +- HashAggregate(isMerge=[false], groupBy=[a1, c1, d1, $e], auxGrouping=[b1], select=[a1, c1, d1, $e, b1, COUNT(d1_0) AS EXPR$3])
    +- Exchange(distribution=[hash[a1, c1, d1, $e]])
       +- Expand(projects=[a1, b1, c1, d1, $e, d1_0], projects=[{a1, b1, c1, null AS d1, 1 AS $e, d1 AS d1_0}, {a1, b1, null AS c1, d1, 2 AS $e, d1 AS d1_0}])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -573,7 +573,7 @@ LogicalAggregate(group=[{0, 1, 2}])
 HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1], select=[a1, b1, c1])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, c1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -594,7 +594,7 @@ HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(coun
 +- Exchange(distribution=[hash[a3, b3]])
    +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0])
       +- Calc(select=[a3, b3, c3])
-         +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -615,7 +615,7 @@ Calc(select=[a1, b1, c1, EXPR$3])
    +- Exchange(distribution=[hash[a1, b1, c1, $e]])
       +- LocalHashAggregate(groupBy=[a1, b1, c1, $e], select=[a1, b1, c1, $e, Partial_COUNT(d1) AS count$0])
          +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}, {a1, null AS b1, null AS c1, d1, 3 AS $e}, {null AS a1, null AS b1, null AS c1, d1, 7 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -636,7 +636,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1, d1, m], select
 +- SortAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1, c1], select=[a1, b1, c1, COUNT(d1) AS d1, MAX(d1) AS m])
    +- Sort(orderBy=[a1 ASC])
       +- Exchange(distribution=[hash[a1]])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -657,7 +657,7 @@ HashAggregate(isMerge=[false], groupBy=[a3, b3], auxGrouping=[c, s, a], select=[
 +- HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(count$0) AS c, Final_SUM(sum$1) AS s, Final_AVG(sum$2, count$3) AS a])
    +- Exchange(distribution=[hash[a3, b3]])
       +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0, Partial_SUM(d3) AS sum$1, Partial_AVG(d3) AS (sum$2, count$3)])
-         +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -677,7 +677,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)], EXPR$2=[SUM(DISTINCT
 HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(b1) AS EXPR$1, SUM(b1) AS EXPR$2])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -698,7 +698,7 @@ HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[d1], select=[a1, d1, C
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, c1, d1, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 2) AS $g_2, =(CASE(=($e, 2:BIGINT), 2:BIGINT, 4:BIGINT), 4) AS $g_4])
       +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, null AS c1, d1, 2 AS $e}, {a1, null AS b1, c1, d1, 4 AS $e}])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -716,7 +716,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM(DISTINCT $1)], EXPR$2=[MAX($1)], EXPR$
     <Resource name="planAfter">
       <![CDATA[
 Calc(select=[a1, b1 AS EXPR$1, b1 AS EXPR$2, c1 AS EXPR$3])
-+- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -738,7 +738,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_1 AS
    +- Sort(orderBy=[a1 ASC])
       +- Exchange(distribution=[hash[a1]])
          +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -761,7 +761,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[$f0]])
       +- LocalHashAggregate(groupBy=[$f0], select=[$f0, Partial_COUNT(c1) AS count$0])
          +- Calc(select=[1 AS $f0, c1])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -782,7 +782,7 @@ HashAggregate(isMerge=[true], groupBy=[a3, b3], select=[a3, b3, Final_COUNT(coun
 +- Exchange(distribution=[hash[a3, b3]])
    +- LocalHashAggregate(groupBy=[a3, b3], select=[a3, b3, Partial_COUNT(c3) AS count$0])
       +- Calc(select=[a3, b3, c3])
-         +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3, d3)]]], fields=[a3, b3, c3, d3])
 ]]>
     </Resource>
   </TestCase>
@@ -802,7 +802,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT($2)])
 HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, c1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -821,7 +821,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[AVG($2)])
       <![CDATA[
 HashAggregate(isMerge=[false], groupBy=[b2], auxGrouping=[c2], select=[b2, c2, AVG(a2) AS EXPR$2])
 +- Exchange(distribution=[hash[b2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 ]]>
     </Resource>
   </TestCase>
@@ -842,7 +842,7 @@ LogicalProject(a1=[$0], b1=[$1], EXPR$2=[$4])
 HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, c1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -862,7 +862,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT(DISTINCT $1)])
 HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(c1) AS EXPR$1])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, c1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -882,7 +882,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT(DISTINCT $2)])
 HashAggregate(isMerge=[false], groupBy=[a1], auxGrouping=[b1], select=[a1, b1, COUNT(c1) AS EXPR$2])
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, c1])
-      +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -903,7 +903,7 @@ HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(b1) FILTER $g_0 A
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, b1, b1_0 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
       +- Expand(projects=[a1, b1, c1, d1, $e, b1_0], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -925,7 +925,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(c1) FILTER $g_1 AS
    +- Exchange(distribution=[hash[a1]])
       +- Calc(select=[a1, b1, c1, b1_0 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
          +- Expand(projects=[a1, b1, c1, d1, $e, b1_0], projects=[{a1, b1, c1, d1, 0 AS $e, b1 AS b1_0}, {a1, null AS b1, c1, d1, 1 AS $e, b1 AS b1_0}])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -946,7 +946,7 @@ HashAggregate(isMerge=[false], groupBy=[a1], select=[a1, COUNT(c1) FILTER $g_0 A
 +- Exchange(distribution=[hash[a1]])
    +- Calc(select=[a1, c1, b1 AS EXPR$2, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
       +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}])
-         +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
@@ -968,7 +968,7 @@ SortAggregate(isMerge=[false], groupBy=[a1], select=[a1, MIN(d1) FILTER $g_1 AS
    +- Exchange(distribution=[hash[a1]])
       +- Calc(select=[a1, c1, d1, b1 AS EXPR$3, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 0) AS $g_0, =(CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT), 1) AS $g_1])
          +- Expand(projects=[a1, b1, c1, d1, $e], projects=[{a1, b1, c1, d1, 0 AS $e}, {a1, b1, null AS c1, d1, 1 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1, d1)]]], fields=[a1, b1, c1, d1])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml
index a6b15d5..dadee2d 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.xml
@@ -39,7 +39,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3])
                   +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0, Partial_SUM(b_0) AS sum$1])
                      +- Expand(projects=[a, b, $e, b_0], projects=[{a, b, 0 AS $e, b AS b_0}, {a, null AS b, 1 AS $e, b AS b_0}])
                         +- Calc(select=[a, b])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -66,7 +66,7 @@ Calc(select=[EXPR$0, EXPR$1])
                +- Exchange(distribution=[hash[a, b, c, $e]])
                   +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e])
                      +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {null AS a, b, c, 4 AS $e}])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -92,7 +92,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM(
                +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1])
                   +- Expand(projects=[a, b, $e, a_0], projects=[{a, null AS b, 1 AS $e, a AS a_0}, {null AS a, b, 2 AS $e, a AS a_0}, {null AS a, null AS b, 3 AS $e, a AS a_0}])
                      +- Calc(select=[a, b])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -119,7 +119,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2, EXPR$3])
                +- Exchange(distribution=[hash[a, b, c, $e]])
                   +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_MAX(a_0) AS max$0, Partial_MIN(a_0) AS min$1])
                      +- Expand(projects=[a, b, c, $e, a_0], projects=[{a, null AS b, c, 2 AS $e, a AS a_0}, {null AS a, b, c, 4 AS $e, a AS a_0}, {null AS a, null AS b, c, 6 AS $e, a AS a_0}])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -146,7 +146,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM(
                +- LocalHashAggregate(groupBy=[a, b, c, $f3, $e], select=[a, b, c, $f3, $e])
                   +- Expand(projects=[a, b, c, $f3, $e], projects=[{a, null AS b, null AS c, null AS $f3, 7 AS $e}, {null AS a, b, null AS c, null AS $f3, 11 AS $e}, {null AS a, null AS b, c, $f3, 12 AS $e}])
                      +- Calc(select=[a, b, c, IS TRUE(>(a, 5)) AS $f3])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -175,7 +175,7 @@ Calc(select=[EXPR$0, EXPR$1])
                      +- Expand(projects=[c, d, a, b, $e, $e_0], projects=[{c, d, a, null AS b, $e, 2 AS $e_0}, {c, d, null AS a, b, $e, 4 AS $e_0}])
                         +- Expand(projects=[c, d, a, b, $e], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}])
                            +- Calc(select=[c, d, a, b])
-                              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -202,7 +202,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_SUM(
                +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_MAX(a_0) AS max$0])
                   +- Expand(projects=[a, $f1, $e, a_0], projects=[{a, $f1, 0 AS $e, a AS a_0}, {a, null AS $f1, 1 AS $e, a AS a_0}, {null AS a, null AS $f1, 3 AS $e, a AS a_0}])
                      +- Calc(select=[a, IS TRUE(>(b, 0)) AS $f1])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -229,7 +229,7 @@ Calc(select=[EXPR$0, EXPR$1, EXPR$2])
                +- LocalHashAggregate(groupBy=[b, c, a, $e], select=[b, c, a, $e])
                   +- Expand(projects=[b, c, a, $e], projects=[{b, null AS c, a, 1 AS $e}, {null AS b, c, a, 2 AS $e}])
                      +- Calc(select=[b, c, a])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -257,7 +257,7 @@ Calc(select=[EXPR$0, EXPR$1, CASE(IS NOT NULL(EXPR$2), EXPR$2, 0) AS EXPR$2])
                   +- LocalHashAggregate(groupBy=[a, $f1, b, $e], select=[a, $f1, b, $e, Partial_COUNT(c) AS count$0])
                      +- Expand(projects=[a, $f1, b, c, $e], projects=[{a, $f1, null AS b, c, 1 AS $e}, {null AS a, null AS $f1, b, c, 6 AS $e}, {null AS a, null AS $f1, null AS b, c, 7 AS $e}])
                         +- Calc(select=[a, IS TRUE(>(c, 0)) AS $f1, b, c])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -285,7 +285,7 @@ Calc(select=[EXPR$0, EXPR$1, CAST(EXPR$2) AS EXPR$2])
                   +- LocalHashAggregate(groupBy=[a, b, d, e, $e, $e_0], select=[a, b, d, e, $e, $e_0, Partial_COUNT(c) AS count$0])
                      +- Expand(projects=[a, b, c, d, e, $e, $e_0], projects=[{a, null AS b, c, d, e, $e, 8 AS $e_0}, {null AS a, b, c, d, e, $e, 16 AS $e_0}, {null AS a, null AS b, c, d, e, $e, 24 AS $e_0}])
                         +- Expand(projects=[a, b, c, d, e, $e], projects=[{a, b, c, d, null AS e, 1 AS $e}, {a, b, c, null AS d, e, 2 AS $e}])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -315,7 +315,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_MIN(min$0) AS EXPR$1
                      +- Exchange(distribution=[hash[d, c, $f4, b, $f6, $e]])
                         +- Expand(projects=[d, e, $f2, c, $f4, b, $f6, $e], projects=[{d, e, $f2, c, $f4, null AS b, null AS $f6, 3 AS $e}, {d, e, $f2, c, null AS $f4, null AS b, null AS $f6, 7 AS $e}, {d, e, $f2, null AS c, null AS $f4, b, $f6, 12 AS $e}, {d, e, $f2, null AS c, null AS $f4, null AS b, null AS $f6, 15 AS $e}])
                            +- Calc(select=[d, e, IS TRUE(<(a, 10)) AS $f2, c, IS TRUE(>(a, 5)) AS $f4, b, IS TRUE(>(b, 3)) AS $f6])
-                              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -342,7 +342,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX
                +- LocalHashAggregate(groupBy=[d, c, $f2, b, $f4, $e], select=[d, c, $f2, b, $f4, $e])
                   +- Expand(projects=[d, c, $f2, b, $f4, $e], projects=[{d, c, $f2, null AS b, null AS $f4, 3 AS $e}, {d, c, null AS $f2, null AS b, null AS $f4, 7 AS $e}, {d, null AS c, null AS $f2, b, $f4, 12 AS $e}])
                      +- Calc(select=[d, c, IS TRUE(>(a, 0)) AS $f2, b, IS TRUE(>(b, 1)) AS $f4])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -369,7 +369,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX
                +- LocalHashAggregate(groupBy=[d, c, $f2, $f3, $e], select=[d, c, $f2, $f3, $e])
                   +- Expand(projects=[d, c, $f2, $f3, $e], projects=[{d, c, $f2, null AS $f3, 1 AS $e}, {d, c, null AS $f2, $f3, 2 AS $e}, {d, c, null AS $f2, null AS $f3, 3 AS $e}])
                      +- Calc(select=[d, c, IS TRUE(>(a, 10)) AS $f2, IS TRUE(<(a, 10)) AS $f3])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -398,7 +398,7 @@ SortAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX
                      +- Exchange(distribution=[hash[d, c, $f2, $e]])
                         +- Expand(projects=[d, c, $f2, e, $e], projects=[{d, c, $f2, e, 0 AS $e}, {d, c, null AS $f2, e, 1 AS $e}, {d, null AS c, null AS $f2, e, 3 AS $e}])
                            +- Calc(select=[d, c, IS TRUE(>(a, 0)) AS $f2, e])
-                              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -422,7 +422,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0])
          +- Exchange(distribution=[hash[a]])
             +- LocalHashAggregate(groupBy=[a], select=[a])
                +- Calc(select=[a])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -448,7 +448,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_MIN(
                +- LocalHashAggregate(groupBy=[a, $f1, $e], select=[a, $f1, $e, Partial_SUM(b) AS sum$0])
                   +- Expand(projects=[a, $f1, b, $e], projects=[{a, $f1, b, 0 AS $e}, {null AS a, null AS $f1, b, 3 AS $e}])
                      +- Calc(select=[a, IS TRUE(>(a, 0)) AS $f1, b])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -475,7 +475,7 @@ Calc(select=[CASE(IS NOT NULL(EXPR$0), EXPR$0, 0) AS EXPR$0, EXPR$1])
                   +- LocalHashAggregate(groupBy=[b, $e], select=[b, $e, Partial_COUNT(a) FILTER $f1 AS count$0])
                      +- Expand(projects=[a, $f1, b, $e], projects=[{a, $f1, b, 0 AS $e}, {a, $f1, null AS b, 1 AS $e}])
                         +- Calc(select=[a, IS TRUE(>(a, 0)) AS $f1, b])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -504,7 +504,7 @@ Calc(select=[EXPR$0, EXPR$1])
                      +- Expand(projects=[b, c, a, $e, b_0, $e_0], projects=[{b, c, a, $e, b_0, 0 AS $e_0}, {b, c, null AS a, $e, b_0, 2 AS $e_0}])
                         +- Expand(projects=[b, c, a, $e, b_0], projects=[{b, null AS c, a, 1 AS $e, b AS b_0}, {null AS b, c, a, 2 AS $e, b AS b_0}])
                            +- Calc(select=[b, c, a])
-                              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -533,7 +533,7 @@ Calc(select=[CAST(EXPR$0) AS EXPR$0, EXPR$1])
                      +- Expand(projects=[c, d, a, b, $e, $e_0], projects=[{c, d, a, b, $e, 0 AS $e_0}, {c, d, a, null AS b, $e, 2 AS $e_0}])
                         +- Expand(projects=[c, d, a, b, $e], projects=[{c, null AS d, a, b, 1 AS $e}, {null AS c, d, a, b, 2 AS $e}])
                            +- Calc(select=[c, d, a, b])
-                              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -560,7 +560,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3])
                   +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(*) AS count1$0])
                      +- Expand(projects=[a, b, $e], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}])
                         +- Calc(select=[a, b])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -587,7 +587,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2])
                   +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(a) AS count$0])
                      +- Expand(projects=[a, b, $e], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}])
                         +- Calc(select=[a, b])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -614,7 +614,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2])
                   +- LocalHashAggregate(groupBy=[a, b, $e], select=[a, b, $e, Partial_COUNT(*) AS count1$0])
                      +- Expand(projects=[a, b, $e], projects=[{a, b, 0 AS $e}, {a, null AS b, 1 AS $e}])
                         +- Calc(select=[a, b])
-                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -638,7 +638,7 @@ HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS EX
          +- Exchange(distribution=[hash[d, $f2, i$c]])
             +- LocalHashAggregate(groupBy=[d, $f2, i$c], select=[d, $f2, i$c])
                +- Calc(select=[d, IS TRUE(>(a, 0)) AS $f2, CASE(IS TRUE(>(a, 0)), c, null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE") AS i$c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -663,7 +663,7 @@ Calc(select=[a, CAST(EXPR$1) AS EXPR$1, EXPR$2, EXPR$3])
                +- Exchange(distribution=[hash[a, b, c, $e]])
                   +- LocalHashAggregate(groupBy=[a, b, c, $e], select=[a, b, c, $e, Partial_COUNT(*) AS count1$0])
                      +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml
index 2af1a76..69fadb8 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.xml
@@ -39,7 +39,7 @@ Calc(select=[d, c, g])
       +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[d, g, $e], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, g, 2 AS $e}, {null AS d, null AS g, 3 AS $e}])
             +- Calc(select=[MOD(deptno, 20) AS d, gender AS g])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -63,7 +63,7 @@ Calc(select=[c])
       +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[$f0, $e], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}])
             +- Calc(select=[1 AS $f0])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -90,7 +90,7 @@ Calc(select=[d1, -(d1, 1) AS d0, c])
       +- LocalHashAggregate(groupBy=[d1, $e], select=[d1, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[d1, $e], projects=[{d1, 0 AS $e}, {null AS d1, 1 AS $e}])
             +- Calc(select=[+(deptno, 1) AS d1])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -117,7 +117,7 @@ Calc(select=[d, c, g])
       +- LocalHashAggregate(groupBy=[d, g, $e], select=[d, g, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[d, g, $e], projects=[{d, g, 0 AS $e}, {d, null AS g, 1 AS $e}, {null AS d, null AS g, 3 AS $e}])
             +- Calc(select=[MOD(deptno, 20) AS d, gender AS g])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -141,7 +141,7 @@ Calc(select=[c])
       +- LocalHashAggregate(groupBy=[$f0, $e], select=[$f0, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[$f0, $e], projects=[{$f0, 0 AS $e}, {null AS $f0, 1 AS $e}])
             +- Calc(select=[1 AS $f0])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -164,7 +164,7 @@ Calc(select=[deptno, c])
       +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
             +- Calc(select=[deptno])
-               +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
 ]]>
     </Resource>
   </TestCase>
@@ -197,7 +197,7 @@ Union(all=[true], union=[deptno, g, c])
 :        +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
 :           +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
 :              +- Calc(select=[deptno], reuse_id=[1])
-:                 +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
+:                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
 +- Calc(select=[deptno, 1:BIGINT AS g, c])
    +- HashAggregate(isMerge=[true], groupBy=[deptno], select=[deptno, Final_COUNT(count1$0) AS c])
       +- Exchange(distribution=[hash[deptno]])
@@ -236,7 +236,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)),
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -264,7 +264,7 @@ HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
                +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
                   +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
                      +- Calc(select=[deptno, gender])
-                        +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -300,10 +300,10 @@ Calc(select=[deptno, gender, min_name], where=[OR(>($f5, 2), AND(=(gender, _UTF-
                +- Expand(projects=[ename, deptno, gender, deptno0, $e], projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gende [...]
                   +- HashJoin(joinType=[InnerJoin], where=[=(deptno, deptno0)], select=[ename, deptno, gender, deptno0], build=[right])
                      :- Exchange(distribution=[hash[deptno]])
-                     :  +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+                     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
                      +- Exchange(distribution=[hash[deptno]])
                         +- Calc(select=[deptno])
-                           +- TableSourceScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname])
 ]]>
     </Resource>
   </TestCase>
@@ -337,7 +337,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)),
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -368,7 +368,7 @@ Calc(select=[deptno, gender, CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 5:
       +- LocalHashAggregate(groupBy=[gender, deptno, $e], select=[gender, deptno, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[gender, deptno, $e], projects=[{gender, deptno, 0 AS $e}, {gender, null AS deptno, 1 AS $e}, {null AS gender, null AS deptno, 3 AS $e}])
             +- Calc(select=[gender, CAST(10) AS deptno], where=[=(deptno, 10)])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -392,7 +392,7 @@ Calc(select=[+(deptno, 1) AS EXPR$0, c])
       +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
             +- Calc(select=[deptno, gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -430,7 +430,7 @@ Sort(orderBy=[deptno ASC, job ASC, empno ASC])
             +- LocalHashAggregate(groupBy=[deptno, job, empno, ename, $e], select=[deptno, job, empno, ename, $e, Partial_SUM(sal) AS sum$0])
                +- Expand(projects=[deptno, job, empno, ename, sal, $e], projects=[{deptno, job, empno, ename, sal, 0 AS $e}, {deptno, job, null AS empno, null AS ename, sal, 3 AS $e}, {deptno, null AS job, null AS empno, null AS ename, sal, 7 AS $e}, {null AS deptno, null AS job, null AS empno, null AS ename, sal, 15 AS $e}])
                   +- Calc(select=[deptno, job, empno, ename, sal])
-                     +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
 ]]>
     </Resource>
   </TestCase>
@@ -465,7 +465,7 @@ Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) A
          +- Calc(select=[deptno, gender, $e], where=[<=(CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT), CASE(=($e, 0:BIGINT), 0:BIGINT, =($e, 1:BIGINT), 2:BIGINT, 7:BIGINT))])
             +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
                +- Calc(select=[deptno, gender])
-                  +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -498,7 +498,7 @@ Calc(select=[c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT, 1:BIGINT) A
       +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
             +- Calc(select=[deptno, gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -532,7 +532,7 @@ Calc(select=[deptno, gender, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT
       +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
             +- Calc(select=[deptno, gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -563,7 +563,7 @@ Calc(select=[deptno, job, c, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)), 0:BIGINT
       +- LocalHashAggregate(groupBy=[deptno, job, $e], select=[deptno, job, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, job, $e], projects=[{deptno, job, 0 AS $e}, {deptno, null AS job, 1 AS $e}, {null AS deptno, job, 2 AS $e}, {null AS deptno, null AS job, 3 AS $e}])
             +- Calc(select=[deptno, job])
-               +- TableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, scott_emp, source: [TestTableSource(empno, ename, job, mgr, hiredate, sal, comm, deptno)]]], fields=[empno, ename, job, mgr, hiredate, sal, comm, deptno])
 ]]>
     </Resource>
   </TestCase>
@@ -586,7 +586,7 @@ Calc(select=[c, 0:BIGINT AS g])
    +- Exchange(distribution=[hash[deptno]])
       +- LocalHashAggregate(groupBy=[deptno], select=[deptno, Partial_COUNT(*) AS count1$0])
          +- Calc(select=[deptno])
-            +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -612,7 +612,7 @@ Calc(select=[c, deptno, CASE(=($e, 0:BIGINT), 0:BIGINT, 1:BIGINT) AS g])
       +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
             +- Calc(select=[deptno])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -638,7 +638,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g])
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -660,7 +660,7 @@ Calc(select=[b, c, a])
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -695,7 +695,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(=($e, 1:BIGINT), 0:BIGINT, 1:BIGINT) A
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1), Partial_COUNT(*) AS count1$2])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, null AS c, 1 AS $e}, {a, null AS b, c, 2 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -720,7 +720,7 @@ Calc(select=[EXPR$0, c])
       +- LocalHashAggregate(groupBy=[EXPR$0, $e], select=[EXPR$0, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[EXPR$0, $e], projects=[{EXPR$0, 0 AS $e}, {null AS EXPR$0, 1 AS $e}])
             +- Calc(select=[+(deptno, 1) AS EXPR$0])
-               +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
 ]]>
     </Resource>
   </TestCase>
@@ -751,7 +751,7 @@ Calc(select=[c])
                +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
                   +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
                      +- Calc(select=[deptno])
-                        +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -781,7 +781,7 @@ Calc(select=[deptno, a, 0:BIGINT AS g, 0:BIGINT AS gb, 0:BIGINT AS gib])
    +- Exchange(distribution=[hash[deptno]])
       +- LocalHashAggregate(groupBy=[deptno], select=[deptno, Partial_AVG(age) AS (sum$0, count$1)])
          +- Calc(select=[deptno, age])
-            +- TableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, emps, source: [TestTableSource(empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat)]]], fields=[empno, name, deptno, gender, city, empid, age, slacker, manager, joinedat])
 ]]>
     </Resource>
   </TestCase>
@@ -815,7 +815,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)),
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -849,7 +849,7 @@ Calc(select=[b, c, a, 0:BIGINT AS g, CASE(OR(=($e, 0:BIGINT), =($e, 1:BIGINT)),
    +- Exchange(distribution=[hash[b, c, $e]])
       +- LocalHashAggregate(groupBy=[b, c, $e], select=[b, c, $e, Partial_AVG(a) AS (sum$0, count$1)])
          +- Expand(projects=[a, b, c, $e], projects=[{a, b, c, 0 AS $e}, {a, b, null AS c, 1 AS $e}, {a, null AS b, null AS c, 3 AS $e}])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -873,7 +873,7 @@ Calc(select=[deptno, c])
       +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, gender, 2 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
             +- Calc(select=[deptno, gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -900,7 +900,7 @@ Calc(select=[+($f0, 1) AS half1, c])
       +- LocalHashAggregate(groupBy=[$f0, gender, $f2, $e], select=[$f0, gender, $f2, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[$f0, gender, $f2, $e], projects=[{$f0, gender, $f2, 0 AS $e}, {$f0, gender, null AS $f2, 1 AS $e}, {$f0, null AS gender, $f2, 2 AS $e}, {$f0, null AS gender, null AS $f2, 3 AS $e}, {null AS $f0, null AS gender, $f2, 6 AS $e}, {null AS $f0, null AS gender, null AS $f2, 7 AS $e}])
             +- Calc(select=[/(deptno, 2) AS $f0, gender, SUBSTRING(ename, 1, 1) AS $f2])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -926,7 +926,7 @@ Calc(select=[gender, +(deptno, 1) AS EXPR$1, c])
       +- LocalHashAggregate(groupBy=[deptno, gender, $e], select=[deptno, gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, gender, $e], projects=[{deptno, gender, 0 AS $e}, {deptno, null AS gender, 1 AS $e}, {null AS deptno, null AS gender, 3 AS $e}])
             +- Calc(select=[deptno, gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -949,7 +949,7 @@ Calc(select=[gender, c])
       +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[gender, $e], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}])
             +- Calc(select=[gender])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -975,7 +975,7 @@ Sort(orderBy=[c DESC])
             +- LocalHashAggregate(groupBy=[gender, $e], select=[gender, $e, Partial_COUNT(*) AS count1$0])
                +- Expand(projects=[gender, $e], projects=[{gender, 0 AS $e}, {null AS gender, 1 AS $e}])
                   +- Calc(select=[gender])
-                     +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -1002,7 +1002,7 @@ Calc(select=[+(deptno, 1) AS d1, c], where=[>(c, 3)])
       +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
             +- Calc(select=[deptno])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
@@ -1026,7 +1026,7 @@ Calc(select=[+(deptno, 1) AS EXPR$0, c])
       +- LocalHashAggregate(groupBy=[deptno, $e], select=[deptno, $e, Partial_COUNT(*) AS count1$0])
          +- Expand(projects=[deptno, $e], projects=[{deptno, 0 AS $e}, {null AS deptno, 1 AS $e}])
             +- Calc(select=[deptno])
-               +- TableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml
index 73f0151..a9daeed 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml
@@ -32,7 +32,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)])
 HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Exchange(distribution=[hash[b]])
    +- Calc(select=[b, a])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -52,7 +52,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)])
 HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Exchange(distribution=[hash[b]])
    +- Calc(select=[b, a])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -72,7 +72,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[var_sum($1)])
 HashAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Exchange(distribution=[hash[b]])
    +- Calc(select=[b, a])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -93,7 +93,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -113,7 +113,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- Calc(select=[a, b, c], where=[=(a, 1)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -134,7 +134,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -156,7 +156,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4])
       +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -177,7 +177,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2, SUM($f3) AS EXPR$3])
 +- Exchange(distribution=[single])
    +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -199,7 +199,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4])
       +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -220,7 +220,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
 +- Exchange(distribution=[hash[a, b]])
    +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
       +- Calc(select=[a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -240,7 +240,7 @@ LogicalAggregate(group=[{0, 1}])
 HashAggregate(isMerge=[false], groupBy=[a, b], select=[a, b])
 +- Exchange(distribution=[hash[a, b]])
    +- Calc(select=[a, b])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -261,7 +261,7 @@ HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
 +- Exchange(distribution=[hash[a, b]])
    +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
       +- Calc(select=[a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -280,7 +280,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -298,7 +298,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
       <![CDATA[
 HashAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -317,7 +317,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- LocalHashAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -348,7 +348,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)]
    +- LocalHashAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -378,7 +378,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[AVG($1)], EXPR$2=[AVG($2)
 HashAggregate(isMerge=[false], select=[AVG(byte) AS EXPR$0, AVG(short) AS EXPR$1, AVG(int) AS EXPR$2, AVG(long) AS EXPR$3, AVG(float) AS EXPR$4, AVG(double) AS EXPR$5, AVG(decimal3020) AS EXPR$6, AVG(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 6) EXPR$7)]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -409,7 +409,7 @@ HashAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)]
    +- LocalHashAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -444,7 +444,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
 HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXP [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)]
    +- LocalHashAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[Rec [...]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -478,7 +478,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
       <![CDATA[
 HashAggregate(isMerge=[false], select=[COUNT(byte) AS EXPR$0, COUNT(short) AS EXPR$1, COUNT(int) AS EXPR$2, COUNT(long) AS EXPR$3, COUNT(float) AS EXPR$4, COUNT(double) AS EXPR$5, COUNT(decimal3020) AS EXPR$6, COUNT(decimal105) AS EXPR$7, COUNT(boolean) AS EXPR$8, COUNT(date) AS EXPR$9, COUNT(time) AS EXPR$10, COUNT(timestamp) AS EXPR$11, COUNT(string) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6 [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) tim [...]
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
 ]]>
     </Resource>
   </TestCase>
@@ -513,7 +513,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
 HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXP [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)]
    +- LocalHashAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[Rec [...]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -532,7 +532,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)])
 HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1, Final_COUNT(count$1) AS EXPR$2])
 +- Exchange(distribution=[hash[a]])
    +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -553,7 +553,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)]
    +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)]
       +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -573,7 +573,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$
 HashAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)]
 +- Exchange(distribution=[single]), rowType=[RecordType(INTEGER $f0)]
    +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -594,7 +594,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)]
    +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)]
       +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -612,7 +612,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)])
       <![CDATA[
 HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[hash[a]])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -631,7 +631,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)])
 HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1, Final_COUNT(count$1) AS EXPR$2])
 +- Exchange(distribution=[hash[a]])
    +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -652,7 +652,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
 +- Exchange(distribution=[hash[a]])
    +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -672,7 +672,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)])
 HashAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[hash[a]])
    +- Calc(select=[a, b, c], where=[=(a, 1)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -693,7 +693,7 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
 +- Exchange(distribution=[hash[a]])
    +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -718,7 +718,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0])
          +- Calc(select=[a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -742,7 +742,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
 +- HashAggregate(isMerge=[false], groupBy=[a], select=[a, MAX(b) AS EXPR$1])
    +- Exchange(distribution=[hash[a]])
       +- Calc(select=[a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -767,7 +767,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0])
          +- Calc(select=[a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -802,7 +802,7 @@ HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)]
    +- LocalHashAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -836,7 +836,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[MAX($1)], EXPR$2=[MAX($2)
 HashAggregate(isMerge=[false], select=[MAX(byte) AS EXPR$0, MAX(short) AS EXPR$1, MAX(int) AS EXPR$2, MAX(long) AS EXPR$3, MAX(float) AS EXPR$4, MAX(double) AS EXPR$5, MAX(decimal3020) AS EXPR$6, MAX(decimal105) AS EXPR$7, MAX(boolean) AS EXPR$8, MAX(date) AS EXPR$9, MAX(time) AS EXPR$10, MAX(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$ [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -871,7 +871,7 @@ HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)]
    +- LocalHashAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -902,7 +902,7 @@ HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
    +- LocalHashAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -937,7 +937,7 @@ HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)]
    +- LocalHashAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -972,7 +972,7 @@ HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)]
    +- LocalHashAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1002,7 +1002,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)
 HashAggregate(isMerge=[false], select=[SUM(byte) AS EXPR$0, SUM(short) AS EXPR$1, SUM(int) AS EXPR$2, SUM(long) AS EXPR$3, SUM(float) AS EXPR$4, SUM(double) AS EXPR$5, SUM(decimal3020) AS EXPR$6, SUM(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 5) EXPR$7)]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1036,7 +1036,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)], EXPR$1=[MIN($1)], EXPR$2=[MIN($2)
 HashAggregate(isMerge=[false], select=[MIN(byte) AS EXPR$0, MIN(short) AS EXPR$1, MIN(int) AS EXPR$2, MIN(long) AS EXPR$3, MIN(float) AS EXPR$4, MIN(double) AS EXPR$5, MIN(decimal3020) AS EXPR$6, MIN(decimal105) AS EXPR$7, MIN(boolean) AS EXPR$8, MIN(date) AS EXPR$9, MIN(time) AS EXPR$10, MIN(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$ [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1067,7 +1067,7 @@ HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
    +- LocalHashAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml
index b185edb..10d3de7 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.xml
@@ -45,7 +45,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
                +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, RANK(*) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2])
                   +- Sort(orderBy=[b ASC, a ASC])
                      +- Exchange(distribution=[hash[b]])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -81,7 +81,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
                         +- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w0$o2, $SUM0(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0])
                            +- Sort(orderBy=[b ASC])
                               +- Exchange(distribution=[single])
-                                 +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -109,7 +109,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
          +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1])
             +- Sort(orderBy=[b ASC, a ASC])
                +- Exchange(distribution=[hash[b]])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -139,7 +139,7 @@ Calc(select=[w0$o0 AS EXPR$0, CASE(>(w1$o0, 0:BIGINT), w1$o1, null:INTEGER) AS E
       +- Sort(orderBy=[c ASC, a ASC])
          +- Exchange(distribution=[hash[c]])
             +- Calc(select=[a, c])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -164,7 +164,7 @@ Calc(select=[c, w0$o0 AS $1])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -189,7 +189,7 @@ Calc(select=[c, w0$o0 AS $1])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -214,7 +214,7 @@ Calc(select=[c, w0$o0 AS $1])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -239,7 +239,7 @@ Calc(select=[w0$o0 AS $0])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -259,7 +259,7 @@ Calc(select=[c, w0$o0 AS $1])
 +- OverAggregate(partitionBy=[c], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, b, c, w0$o0])
    +- Sort(orderBy=[c ASC])
       +- Exchange(distribution=[hash[c]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -279,7 +279,7 @@ Calc(select=[c, CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$1])
 +- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1])
    +- Sort(orderBy=[b ASC])
       +- Exchange(distribution=[single])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -298,7 +298,7 @@ LogicalProject(c=[$2], EXPR$1=[COUNT() OVER (RANGE BETWEEN UNBOUNDED PRECEDING A
 Calc(select=[c, w0$o0 AS $1])
 +- OverAggregate(window#0=[COUNT(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, b, c, w0$o0])
    +- Exchange(distribution=[single])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -323,7 +323,7 @@ Calc(select=[w0$o0 AS $0])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -348,7 +348,7 @@ Calc(select=[w0$o0 AS $0])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -368,7 +368,7 @@ Calc(select=[w0$o0 AS $0])
 +- OverAggregate(partitionBy=[c], orderBy=[a ASC, c ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0])
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -387,7 +387,7 @@ LogicalProject(EXPR$0=[COUNT(1) OVER (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBO
 OverAggregate(window#0=[COUNT(1) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[w0$o0])
 +- Exchange(distribution=[single])
    +- Calc(select=[])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -413,7 +413,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS E
    +- Sort(orderBy=[c ASC, a ASC])
       +- Exchange(distribution=[hash[c]])
          +- Calc(select=[a, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -444,7 +444,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
          +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o1, $SUM0(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o1, w0$o2])
             +- Sort(orderBy=[b ASC, a ASC])
                +- Exchange(distribution=[hash[b]])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -477,7 +477,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
                +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[MAX(a) AS w2$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w2$o0, w0$o0])
                   +- Sort(orderBy=[b ASC, a ASC])
                      +- Exchange(distribution=[hash[b]])
-                        +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -503,7 +503,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS E
    +- Sort(orderBy=[b ASC, a ASC])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -534,7 +534,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS E
          +- Sort(orderBy=[b ASC, a ASC])
             +- Exchange(distribution=[hash[b]])
                +- Calc(select=[a, b])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -561,7 +561,7 @@ Calc(select=[w0$o0 AS $0, w1$o0 AS $1])
       +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0])
          +- Sort(orderBy=[b ASC, a ASC])
             +- Exchange(distribution=[hash[b]])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -590,7 +590,7 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS E
          +- Sort(orderBy=[b ASC])
             +- Exchange(distribution=[hash[b]])
                +- Calc(select=[a, b])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -616,7 +616,7 @@ Calc(select=[a, w0$o0 AS $1, w1$o0 AS $2])
    +- OverAggregate(partitionBy=[b], orderBy=[c ASC, a DESC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0])
       +- Sort(orderBy=[b ASC, c ASC, a DESC])
          +- Exchange(distribution=[hash[b]])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml
index 3f3e03f..54034a0 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml
@@ -33,7 +33,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -54,7 +54,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -75,7 +75,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -96,7 +96,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -116,7 +116,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- Calc(select=[a, b, c], where=[=(a, 1)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -137,7 +137,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -159,7 +159,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4])
       +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -180,7 +180,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2, SUM($f3) AS EXPR$3])
 +- Exchange(distribution=[single])
    +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -202,7 +202,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3, Partial_SUM($f3) AS sum$4])
       +- Calc(select=[CAST(1) AS a, b, c, c._1 AS $f3], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -225,7 +225,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
       +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
          +- Sort(orderBy=[a ASC, b ASC])
             +- Calc(select=[a, b])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -246,7 +246,7 @@ SortAggregate(isMerge=[false], groupBy=[a, b], select=[a, b])
 +- Sort(orderBy=[a ASC, b ASC])
    +- Exchange(distribution=[hash[a, b]])
       +- Calc(select=[a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -269,7 +269,7 @@ SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
       +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
          +- Sort(orderBy=[a ASC, b ASC])
             +- Calc(select=[a, b])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -288,7 +288,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -306,7 +306,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
       <![CDATA[
 SortAggregate(isMerge=[false], select=[AVG(a) AS EXPR$0, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -325,7 +325,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[SUM($1)], EXPR$2=[COUNT($
 SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS EXPR$1, Final_COUNT(count$3) AS EXPR$2])
 +- Exchange(distribution=[single])
    +- LocalSortAggregate(select=[Partial_AVG(a) AS (sum$0, count$1), Partial_SUM(b) AS sum$2, Partial_COUNT(c) AS count$3])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -356,7 +356,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)]
    +- LocalSortAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -386,7 +386,7 @@ LogicalAggregate(group=[{}], EXPR$0=[AVG($0)], EXPR$1=[AVG($1)], EXPR$2=[AVG($2)
 SortAggregate(isMerge=[false], select=[AVG(byte) AS EXPR$0, AVG(short) AS EXPR$1, AVG(int) AS EXPR$2, AVG(long) AS EXPR$3, AVG(float) AS EXPR$4, AVG(double) AS EXPR$5, AVG(decimal3020) AS EXPR$6, AVG(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 6) EXPR$7)]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -417,7 +417,7 @@ SortAggregate(isMerge=[true], select=[Final_AVG(sum$0, count$1) AS EXPR$0, Final
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, BIGINT count$7, DOUBLE sum$8, BIGINT count$9, DOUBLE sum$10, BIGINT count$11, DECIMAL(38, 20) sum$12, BIGINT count$13, DECIMAL(38, 5) sum$14, BIGINT count$15)]
    +- LocalSortAggregate(select=[Partial_AVG(byte) AS (sum$0, count$1), Partial_AVG(short) AS (sum$2, count$3), Partial_AVG(int) AS (sum$4, count$5), Partial_AVG(long) AS (sum$6, count$7), Partial_AVG(float) AS (sum$8, count$9), Partial_AVG(double) AS (sum$10, count$11), Partial_AVG(decimal3020) AS (sum$12, count$13), Partial_AVG(decimal105) AS (sum$14, count$15)]), rowType=[RecordType(BIGINT sum$0, BIGINT count$1, BIGINT sum$2, BIGINT count$3, BIGINT sum$4, BIGINT count$5, BIGINT sum$6, [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -452,7 +452,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
 SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXP [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)]
    +- LocalSortAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[Rec [...]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -486,7 +486,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
       <![CDATA[
 SortAggregate(isMerge=[false], select=[COUNT(byte) AS EXPR$0, COUNT(short) AS EXPR$1, COUNT(int) AS EXPR$2, COUNT(long) AS EXPR$3, COUNT(float) AS EXPR$4, COUNT(double) AS EXPR$5, COUNT(decimal3020) AS EXPR$6, COUNT(decimal105) AS EXPR$7, COUNT(boolean) AS EXPR$8, COUNT(date) AS EXPR$9, COUNT(time) AS EXPR$10, COUNT(timestamp) AS EXPR$11, COUNT(string) AS EXPR$12]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1, BIGINT EXPR$2, BIGINT EXPR$3, BIGINT EXPR$4, BIGINT EXPR$5, BIGINT EXPR$6 [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) tim [...]
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
 ]]>
     </Resource>
   </TestCase>
@@ -521,7 +521,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT($0)], EXPR$1=[COUNT($1)], EXPR$2=[COU
 SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1, Final_COUNT(count$2) AS EXPR$2, Final_COUNT(count$3) AS EXPR$3, Final_COUNT(count$4) AS EXPR$4, Final_COUNT(count$5) AS EXPR$5, Final_COUNT(count$6) AS EXPR$6, Final_COUNT(count$7) AS EXPR$7, Final_COUNT(count$8) AS EXPR$8, Final_COUNT(count$9) AS EXPR$9, Final_COUNT(count$10) AS EXPR$10, Final_COUNT(count$11) AS EXPR$11, Final_COUNT(count$12) AS EXPR$12]), rowType=[RecordType(BIGINT EXP [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count$0, BIGINT count$1, BIGINT count$2, BIGINT count$3, BIGINT count$4, BIGINT count$5, BIGINT count$6, BIGINT count$7, BIGINT count$8, BIGINT count$9, BIGINT count$10, BIGINT count$11, BIGINT count$12)]
    +- LocalSortAggregate(select=[Partial_COUNT(byte) AS count$0, Partial_COUNT(short) AS count$1, Partial_COUNT(int) AS count$2, Partial_COUNT(long) AS count$3, Partial_COUNT(float) AS count$4, Partial_COUNT(double) AS count$5, Partial_COUNT(decimal3020) AS count$6, Partial_COUNT(decimal105) AS count$7, Partial_COUNT(boolean) AS count$8, Partial_COUNT(date) AS count$9, Partial_COUNT(time) AS count$10, Partial_COUNT(timestamp) AS count$11, Partial_COUNT(string) AS count$12]), rowType=[Rec [...]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -542,7 +542,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
    +- Exchange(distribution=[hash[a]])
       +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
          +- Sort(orderBy=[a ASC])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -563,7 +563,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)]
    +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)]
       +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -583,7 +583,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$
 SortAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)]
 +- Exchange(distribution=[single]), rowType=[RecordType(INTEGER $f0)]
    +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -604,7 +604,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType
 +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)]
    +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)]
       +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -623,7 +623,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[COUNT($2)])
 SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c) AS EXPR$2])
 +- Sort(orderBy=[a ASC])
    +- Exchange(distribution=[hash[a]])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -644,7 +644,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
    +- Exchange(distribution=[hash[a]])
       +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
          +- Sort(orderBy=[a ASC])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -667,7 +667,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
       +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
          +- Sort(orderBy=[a ASC])
             +- Calc(select=[a, b, c], where=[=(a, 1)])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -688,7 +688,7 @@ SortAggregate(isMerge=[false], groupBy=[a], select=[a, SUM(b) AS EXPR$1, COUNT(c
 +- Sort(orderBy=[a ASC])
    +- Exchange(distribution=[hash[a]])
       +- Calc(select=[a, b, c], where=[=(a, 1)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -711,7 +711,7 @@ SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS EXPR$1
       +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_COUNT(c) AS count$1])
          +- Sort(orderBy=[a ASC])
             +- Calc(select=[a, b, c], where=[=(a, 1)])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -738,7 +738,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
          +- LocalSortAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0])
             +- Sort(orderBy=[a ASC])
                +- Calc(select=[a, b])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -763,7 +763,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
    +- Sort(orderBy=[a ASC])
       +- Exchange(distribution=[hash[a]])
          +- Calc(select=[a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -790,7 +790,7 @@ Calc(select=[a, EXPR$1, _UTF-16LE'test' AS c])
          +- LocalSortAggregate(groupBy=[a], select=[a, Partial_MAX(b) AS max$0])
             +- Sort(orderBy=[a ASC])
                +- Calc(select=[a, b])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -825,7 +825,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)]
    +- LocalSortAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -859,7 +859,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)], EXPR$1=[MAX($1)], EXPR$2=[MAX($2)
 SortAggregate(isMerge=[false], select=[MAX(byte) AS EXPR$0, MAX(short) AS EXPR$1, MAX(int) AS EXPR$2, MAX(long) AS EXPR$3, MAX(float) AS EXPR$4, MAX(double) AS EXPR$5, MAX(decimal3020) AS EXPR$6, MAX(decimal105) AS EXPR$7, MAX(boolean) AS EXPR$8, MAX(date) AS EXPR$9, MAX(time) AS EXPR$10, MAX(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$ [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -894,7 +894,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0, Final_MAX(max$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DOUBLE max$5, DECIMAL(30, 20) max$6, DECIMAL(10, 5) max$7, BOOLEAN max$8, DATE max$9, TIME(0) max$10, TIMESTAMP(3) max$11)]
    +- LocalSortAggregate(select=[Partial_MAX(byte) AS max$0, Partial_MAX(short) AS max$1, Partial_MAX(int) AS max$2, Partial_MAX(long) AS max$3, Partial_MAX(float) AS max$4, Partial_MAX(double) AS max$5, Partial_MAX(decimal3020) AS max$6, Partial_MAX(decimal105) AS max$7, Partial_MAX(boolean) AS max$8, Partial_MAX(date) AS max$9, Partial_MAX(time) AS max$10, Partial_MAX(timestamp) AS max$11]), rowType=[RecordType(TINYINT max$0, SMALLINT max$1, INTEGER max$2, BIGINT max$3, FLOAT max$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -915,7 +915,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]), rowType=[Rec
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) max$0)]
    +- LocalSortAggregate(select=[Partial_MAX(string) AS max$0]), rowType=[RecordType(VARCHAR(2147483647) max$0)]
       +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -935,7 +935,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MAX($0)]), rowType=[RecordType(VARCHAR(2147
 SortAggregate(isMerge=[false], select=[MAX(string) AS EXPR$0]), rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) string)]
    +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -956,7 +956,7 @@ SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0]), rowType=[Rec
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) max$0)]
    +- LocalSortAggregate(select=[Partial_MAX(string) AS max$0]), rowType=[RecordType(VARCHAR(2147483647) max$0)]
       +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -991,7 +991,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)]
    +- LocalSortAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1025,7 +1025,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)], EXPR$1=[MIN($1)], EXPR$2=[MIN($2)
 SortAggregate(isMerge=[false], select=[MIN(byte) AS EXPR$0, MIN(short) AS EXPR$1, MIN(int) AS EXPR$2, MIN(long) AS EXPR$3, MIN(float) AS EXPR$4, MIN(double) AS EXPR$5, MIN(decimal3020) AS EXPR$6, MIN(decimal105) AS EXPR$7, MIN(boolean) AS EXPR$8, MIN(date) AS EXPR$9, MIN(time) AS EXPR$10, MIN(timestamp) AS EXPR$11]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(30, 20) EXPR$6, DECIMAL(10, 5) EXPR$7, BOOLEAN EXPR$ [...]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1060,7 +1060,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0, Final_MIN(min$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DOUBLE min$5, DECIMAL(30, 20) min$6, DECIMAL(10, 5) min$7, BOOLEAN min$8, DATE min$9, TIME(0) min$10, TIMESTAMP(3) min$11)]
    +- LocalSortAggregate(select=[Partial_MIN(byte) AS min$0, Partial_MIN(short) AS min$1, Partial_MIN(int) AS min$2, Partial_MIN(long) AS min$3, Partial_MIN(float) AS min$4, Partial_MIN(double) AS min$5, Partial_MIN(decimal3020) AS min$6, Partial_MIN(decimal105) AS min$7, Partial_MIN(boolean) AS min$8, Partial_MIN(date) AS min$9, Partial_MIN(time) AS min$10, Partial_MIN(timestamp) AS min$11]), rowType=[RecordType(TINYINT min$0, SMALLINT min$1, INTEGER min$2, BIGINT min$3, FLOAT min$4, DO [...]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105, boolean, date, time, timestamp]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105, BOOLEAN boolean, DATE date, TIME(0) time, TIMESTAMP(3) timestamp)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1081,7 +1081,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) min$0)]
    +- LocalSortAggregate(select=[Partial_MIN(string) AS min$0]), rowType=[RecordType(VARCHAR(2147483647) min$0)]
       +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1101,7 +1101,7 @@ LogicalAggregate(group=[{}], EXPR$0=[MIN($0)]), rowType=[RecordType(VARCHAR(2147
 SortAggregate(isMerge=[false], select=[MIN(string) AS EXPR$0]), rowType=[RecordType(VARCHAR(2147483647) EXPR$0)]
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) string)]
    +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1122,7 +1122,7 @@ SortAggregate(isMerge=[true], select=[Final_MIN(min$0) AS EXPR$0]), rowType=[Rec
 +- Exchange(distribution=[single]), rowType=[RecordType(VARCHAR(2147483647) min$0)]
    +- LocalSortAggregate(select=[Partial_MIN(string) AS min$0]), rowType=[RecordType(VARCHAR(2147483647) min$0)]
       +- Calc(select=[string]), rowType=[RecordType(VARCHAR(2147483647) string)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1153,7 +1153,7 @@ SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
    +- LocalSortAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1174,7 +1174,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1195,7 +1195,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1225,7 +1225,7 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)], EXPR$1=[SUM($1)], EXPR$2=[SUM($2)
 SortAggregate(isMerge=[false], select=[SUM(byte) AS EXPR$0, SUM(short) AS EXPR$1, SUM(int) AS EXPR$2, SUM(long) AS EXPR$3, SUM(float) AS EXPR$4, SUM(double) AS EXPR$5, SUM(decimal3020) AS EXPR$6, SUM(decimal105) AS EXPR$7]), rowType=[RecordType(TINYINT EXPR$0, SMALLINT EXPR$1, INTEGER EXPR$2, BIGINT EXPR$3, FLOAT EXPR$4, DOUBLE EXPR$5, DECIMAL(38, 20) EXPR$6, DECIMAL(38, 5) EXPR$7)]
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
    +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0)  [...]
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TI [...]
 ]]>
     </Resource>
   </TestCase>
@@ -1246,7 +1246,7 @@ SortAggregate(isMerge=[false], groupBy=[b], select=[b, var_sum(a) AS EXPR$1])
 +- Sort(orderBy=[b ASC])
    +- Exchange(distribution=[hash[b]])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1277,7 +1277,7 @@ SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS EXPR$0, Final_SUM(sum$
 +- Exchange(distribution=[single]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
    +- LocalSortAggregate(select=[Partial_SUM(byte) AS sum$0, Partial_SUM(short) AS sum$1, Partial_SUM(int) AS sum$2, Partial_SUM(long) AS sum$3, Partial_SUM(float) AS sum$4, Partial_SUM(double) AS sum$5, Partial_SUM(decimal3020) AS sum$6, Partial_SUM(decimal105) AS sum$7]), rowType=[RecordType(TINYINT sum$0, SMALLINT sum$1, INTEGER sum$2, BIGINT sum$3, FLOAT sum$4, DOUBLE sum$5, DECIMAL(38, 20) sum$6, DECIMAL(38, 5) sum$7)]
       +- Calc(select=[byte, short, int, long, float, double, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)]
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME( [...]
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105)]]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, [...]
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml
index 58fc85d..e13743f 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.xml
@@ -44,7 +44,7 @@ Calc(select=[CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0, CAST(/(-($f0,
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM($f2) AS sum$0, Partial_SUM(b) AS sum$1, Partial_COUNT(b) AS count$2])
          +- Calc(select=[ts, b, *(b, b) AS $f2])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -75,7 +75,7 @@ Calc(select=[CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0, CAST(/(-($f0,
 +- HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[SUM($f2) AS $f0, SUM(b) AS $f1, COUNT(b) AS $f2])
    +- Exchange(distribution=[single])
       +- Calc(select=[ts, b, *(b, b) AS $f2])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -107,7 +107,7 @@ Calc(select=[CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0, CAST(/(-($f0,
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM($f2) AS sum$0, Partial_SUM(b) AS sum$1, Partial_COUNT(b) AS count$2])
          +- Calc(select=[ts, b, *(b, b) AS $f2])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -139,7 +139,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[AND(>($f1, 0), =(EXTRACT(FLAG(QU
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_COUNT(*) AS count1$0, Partial_SUM(a) AS sum$1])
          +- Calc(select=[ts, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -170,7 +170,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[AND(>($f1, 0), =(EXTRACT(FLAG(QU
 +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[COUNT(*) AS EXPR$0, SUM(a) AS $f1])
    +- Exchange(distribution=[single])
       +- Calc(select=[ts, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -202,7 +202,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1], where=[AND(>($f1, 0), =(EXTRACT(FLAG(QU
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 60000, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_COUNT(*) AS count1$0, Partial_SUM(a) AS sum$1])
          +- Calc(select=[ts, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -225,7 +225,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_COUNT(c) AS count$0])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -254,7 +254,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2])
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(a) AS sum$0])
          +- Calc(select=[b, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -282,7 +282,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2])
 +- HashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[SUM(a) AS EXPR$0])
    +- Exchange(distribution=[single])
       +- Calc(select=[b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -311,7 +311,7 @@ Calc(select=[EXPR$0, w$start AS EXPR$1, w$end AS EXPR$2])
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, b, 3000, 3000)], properties=[w$start, w$end, w$rowtime], select=[Partial_SUM(a) AS sum$0])
          +- Calc(select=[b, a])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -333,7 +333,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Partial_AVG(c) AS (sum$0, count$1), Partial_SUM(a) AS sum$2])
       +- Calc(select=[b, c, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -354,7 +354,7 @@ LogicalProject(EXPR$0=[$1], EXPR$1=[$2])
 HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[AVG(c) AS EXPR$0, SUM(a) AS EXPR$1])
 +- Exchange(distribution=[single])
    +- Calc(select=[b, c, a])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -376,7 +376,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Final_AV
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 3000)], select=[Partial_AVG(c) AS (sum$0, count$1), Partial_SUM(a) AS sum$2])
       +- Calc(select=[b, c, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -398,7 +398,7 @@ SortWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec
 +- Sort(orderBy=[ts ASC])
    +- Exchange(distribution=[single])
       +- Calc(select=[ts, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -420,7 +420,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1])
       +- Calc(select=[ts, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -442,7 +442,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1])
       +- Calc(select=[ts, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -463,7 +463,7 @@ LogicalProject(sumA=[$1], cntB=[$2])
 HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[SUM(a) AS sumA, COUNT(b) AS cntB])
 +- Exchange(distribution=[single])
    +- Calc(select=[ts, a, b])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -485,7 +485,7 @@ HashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Fina
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, ts, 7200000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1])
       +- Calc(select=[ts, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -507,7 +507,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a
 +- HashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Final_SUM(sum$0) AS sumA, Final_AVG(sum$1, count$2) AS avgB])
    +- Exchange(distribution=[hash[c, d]])
       +- LocalHashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Partial_SUM(a) AS sum$0, Partial_AVG(b) AS (sum$1, count$2)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -529,7 +529,7 @@ HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], selec
 +- Exchange(distribution=[single])
    +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 5400000, 900000)], select=[Partial_SUM(a) AS sum$0, Partial_COUNT(b) AS count$1])
       +- Calc(select=[ts, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -551,7 +551,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a
 +- SortWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, SUM(a) AS sumA, AVG(b) AS avgB])
    +- Sort(orderBy=[c ASC, d ASC, ts ASC])
       +- Exchange(distribution=[hash[c, d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -573,7 +573,7 @@ Calc(select=[c, w$end AS EXPR$1, w$start AS EXPR$2, w$rowtime AS EXPR$3, sumA, a
 +- HashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Final_SUM(sum$0) AS sumA, Final_AVG(sum$1, count$2) AS avgB])
    +- Exchange(distribution=[hash[c, d]])
       +- LocalHashWindowAggregate(groupBy=[c, d], window=[SlidingGroupWindow('w$, ts, 10800000, 3600000)], properties=[w$start, w$end, w$rowtime], select=[c, d, Partial_SUM(a) AS sum$0, Partial_AVG(b) AS (sum$1, count$2)])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -605,7 +605,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m
    +- Exchange(distribution=[hash[c]])
       +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, Partial_SUM(a) AS sum$0, Partial_MIN(b) AS min$1])
          +- Calc(select=[ts, c, a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -636,7 +636,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m
 +- HashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, SUM(a) AS sumA, MIN(b) AS minB])
    +- Exchange(distribution=[hash[c]])
       +- Calc(select=[ts, c, a, b])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -668,7 +668,7 @@ Calc(select=[w$start AS EXPR$0, w$end AS EXPR$1, w$rowtime AS EXPR$2, c, sumA, m
    +- Exchange(distribution=[hash[c]])
       +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c, Partial_SUM(a) AS sum$0, Partial_MIN(b) AS min$1])
          +- Calc(select=[ts, c, a, b])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -704,7 +704,7 @@ Calc(select=[CAST(CASE(=($f1, 0), null:INTEGER, s)) AS s, CAST(CAST(/(CASE(=($f1
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_$SUM0($f1) AS sum$0, Partial_COUNT(*) AS count1$1])
          +- Calc(select=[b, CASE(=(a, 1), 1, 99) AS $f1])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -739,7 +739,7 @@ Calc(select=[CAST(CASE(=($f1, 0), null:INTEGER, s)) AS s, CAST(CAST(/(CASE(=($f1
 +- HashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[$SUM0($f1) AS s, COUNT(*) AS $f1])
    +- Exchange(distribution=[single])
       +- Calc(select=[b, CASE(=(a, 1), 1, 99) AS $f1])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -775,7 +775,7 @@ Calc(select=[CAST(CASE(=($f1, 0), null:INTEGER, s)) AS s, CAST(CAST(/(CASE(=($f1
    +- Exchange(distribution=[single])
       +- LocalHashWindowAggregate(window=[TumblingGroupWindow('w$, b, 900000)], properties=[w$start, w$end, w$rowtime], select=[Partial_$SUM0($f1) AS sum$0, Partial_COUNT(*) AS count1$1])
          +- Calc(select=[b, CASE(=(a, 1), 1, 99) AS $f1])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -798,7 +798,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[b]])
       +- LocalHashWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[b, Partial_COUNT(c) AS count$0])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -821,7 +821,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[b ASC, ts ASC])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -844,7 +844,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[b]])
       +- LocalHashWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[b, Partial_COUNT(c) AS count$0])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -869,7 +869,7 @@ Calc(select=[EXPR$0])
          +- LocalSortWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_MAX(c) AS max$0])
             +- Sort(orderBy=[a ASC, ts ASC])
                +- Calc(select=[a, ts, c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -892,7 +892,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[a ASC, ts ASC])
       +- Exchange(distribution=[hash[a]])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -917,7 +917,7 @@ Calc(select=[EXPR$0])
          +- LocalSortWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_MAX(c) AS max$0])
             +- Sort(orderBy=[a ASC, ts ASC])
                +- Calc(select=[a, ts, c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -940,7 +940,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[b ASC, ts ASC])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -963,7 +963,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[b ASC, ts ASC])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -986,7 +986,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[b ASC, ts ASC])
       +- Exchange(distribution=[hash[b]])
          +- Calc(select=[b, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1008,7 +1008,7 @@ Calc(select=[EXPR$0])
 +- SortWindowAggregate(groupBy=[a, d], window=[SlidingGroupWindow('w$, b, 3600000, 3000)], select=[a, d, countFun(c) AS EXPR$0])
    +- Sort(orderBy=[a ASC, d ASC, b ASC])
       +- Exchange(distribution=[hash[a, d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1030,7 +1030,7 @@ Calc(select=[EXPR$0])
 +- SortWindowAggregate(groupBy=[a, d], window=[SlidingGroupWindow('w$, b, 3600000, 3000)], select=[a, d, countFun(c) AS EXPR$0])
    +- Sort(orderBy=[a ASC, d ASC, b ASC])
       +- Exchange(distribution=[hash[a, d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1054,7 +1054,7 @@ Calc(select=[EXPR$0])
       +- Exchange(distribution=[hash[a, d]])
          +- LocalSortWindowAggregate(groupBy=[a, d], window=[SlidingGroupWindow('w$, b, 3600000, 3000)], select=[a, d, Partial_countFun(c) AS EXPR$0])
             +- Sort(orderBy=[a ASC, d ASC, b ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1077,7 +1077,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[a ASC, ts ASC])
       +- Exchange(distribution=[hash[a]])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1100,7 +1100,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashWindowAggregate(groupBy=[a], window=[SlidingGroupWindow('w$, ts, 3600000, 3000)], select=[a, Partial_COUNT(c) AS count$0])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1123,7 +1123,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_COUNT(c) AS count$0])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1145,7 +1145,7 @@ Calc(select=[EXPR$0])
 +- HashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, COUNT(c) AS EXPR$0])
    +- Exchange(distribution=[hash[a]])
       +- Calc(select=[a, ts, c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1168,7 +1168,7 @@ Calc(select=[EXPR$0])
    +- Exchange(distribution=[hash[a]])
       +- LocalHashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_COUNT(c) AS count$0])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1190,7 +1190,7 @@ Calc(select=[EXPR$0, EXPR$1])
 +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Final_AVG(sum$0, count$1) AS EXPR$0, Final_COUNT(count$2) AS EXPR$1])
    +- Exchange(distribution=[hash[a, d]])
       +- LocalHashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_COUNT(a) AS count$2])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1211,7 +1211,7 @@ LogicalProject(EXPR$0=[$3], EXPR$1=[$4])
 Calc(select=[EXPR$0, EXPR$1])
 +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, AVG(c) AS EXPR$0, COUNT(a) AS EXPR$1])
    +- Exchange(distribution=[hash[a, d]])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1233,7 +1233,7 @@ Calc(select=[EXPR$0, EXPR$1])
 +- HashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Final_AVG(sum$0, count$1) AS EXPR$0, Final_COUNT(count$2) AS EXPR$1])
    +- Exchange(distribution=[hash[a, d]])
       +- LocalHashWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_COUNT(a) AS count$2])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1258,7 +1258,7 @@ Calc(select=[EXPR$0])
          +- LocalSortWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_MAX(c) AS max$0])
             +- Sort(orderBy=[a ASC, ts ASC])
                +- Calc(select=[a, ts, c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1281,7 +1281,7 @@ Calc(select=[EXPR$0])
    +- Sort(orderBy=[a ASC, ts ASC])
       +- Exchange(distribution=[hash[a]])
          +- Calc(select=[a, ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1306,7 +1306,7 @@ Calc(select=[EXPR$0])
          +- LocalSortWindowAggregate(groupBy=[a], window=[TumblingGroupWindow('w$, ts, 3000)], select=[a, Partial_MAX(c) AS max$0])
             +- Sort(orderBy=[a ASC, ts ASC])
                +- Calc(select=[a, ts, c])
-                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(ts, a, b, c)]]], fields=[ts, a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -1328,7 +1328,7 @@ Calc(select=[EXPR$0, EXPR$1])
 +- SortWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, AVG(c) AS EXPR$0, countFun(a) AS EXPR$1])
    +- Sort(orderBy=[a ASC, d ASC, b ASC])
       +- Exchange(distribution=[hash[a, d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1350,7 +1350,7 @@ Calc(select=[EXPR$0, EXPR$1])
 +- SortWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, AVG(c) AS EXPR$0, countFun(a) AS EXPR$1])
    +- Sort(orderBy=[a ASC, d ASC, b ASC])
       +- Exchange(distribution=[hash[a, d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1374,7 +1374,7 @@ Calc(select=[EXPR$0, EXPR$1])
       +- Exchange(distribution=[hash[a, d]])
          +- LocalSortWindowAggregate(groupBy=[a, d], window=[TumblingGroupWindow('w$, b, 3000)], select=[a, d, Partial_AVG(c) AS (sum$0, count$1), Partial_countFun(a) AS EXPR$1])
             +- Sort(orderBy=[a ASC, d ASC, b ASC])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d)]]], fields=[a, b, c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1398,7 +1398,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final
       +- LocalSortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Partial_weightedAvg(b, a) AS wAvg])
          +- Sort(orderBy=[ts ASC])
             +- Calc(select=[ts, b, a])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -1420,7 +1420,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[weigh
 +- Sort(orderBy=[ts ASC])
    +- Exchange(distribution=[single])
       +- Calc(select=[ts, b, a])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -1444,7 +1444,7 @@ SortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Final
       +- LocalSortWindowAggregate(window=[TumblingGroupWindow('w$, ts, 240000)], select=[Partial_weightedAvg(b, a) AS wAvg])
          +- Sort(orderBy=[ts ASC])
             +- Calc(select=[ts, b, a])
-               +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -1489,7 +1489,7 @@ Union(all=[true], union=[EXPR$0])
 :     +- Exchange(distribution=[single])
 :        +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[])
 :           +- Calc(select=[ts], reuse_id=[1])
-:              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 +- Calc(select=[1 AS EXPR$0])
    +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[])
       +- Exchange(distribution=[single])
@@ -1517,7 +1517,7 @@ Calc(select=[w$end AS EXPR$0])
    +- Exchange(distribution=[hash[c]])
       +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c])
          +- Calc(select=[ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -1561,7 +1561,7 @@ Union(all=[true], union=[EXPR$0])
 :  +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[])
 :     +- Exchange(distribution=[single], reuse_id=[1])
 :        +- Calc(select=[ts])
-:           +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 +- Calc(select=[1 AS EXPR$0])
    +- SortWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[])
       +- Sort(orderBy=[ts ASC])
@@ -1587,7 +1587,7 @@ Calc(select=[w$end AS EXPR$0])
 +- HashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c])
    +- Exchange(distribution=[hash[c]])
       +- Calc(select=[ts, c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
@@ -1632,7 +1632,7 @@ Union(all=[true], union=[EXPR$0])
 :     +- Exchange(distribution=[single])
 :        +- LocalHashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 3600000, 3600000)], select=[])
 :           +- Calc(select=[ts], reuse_id=[1])
-:              +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 +- Calc(select=[1 AS EXPR$0])
    +- HashWindowAggregate(window=[SlidingGroupWindow('w$, ts, 7200000, 3600000)], select=[])
       +- Exchange(distribution=[single])
@@ -1660,7 +1660,7 @@ Calc(select=[w$end AS EXPR$0])
    +- Exchange(distribution=[hash[c]])
       +- LocalHashWindowAggregate(groupBy=[c], window=[TumblingGroupWindow('w$, ts, 240000)], properties=[w$start, w$end, w$rowtime], select=[c])
          +- Calc(select=[ts, c])
-            +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c, d, ts)]]], fields=[a, b, c, d, ts])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
index fffdc6f..fc4c6f9 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
@@ -34,9 +34,9 @@ Calc(select=[c, g])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[d, g, a, c], isBroadcast=[true], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -55,9 +55,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftOuterJoin], where=[AND(=(a, d), <(d, 2), <(b, h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -78,8 +78,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
 HashJoin(joinType=[InnerJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[d, e, f, g, h], where=[<(d, 2)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -102,9 +102,9 @@ Calc(select=[c, g])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[d, g, a, c], isBroadcast=[true], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -125,10 +125,10 @@ LogicalProject(c=[$2], g=[$6])
       <![CDATA[
 Calc(select=[c, g])
 +- HashJoin(joinType=[InnerJoin], where=[AND(=(b, e), =(a, d))], select=[a, b, c, d, e, g], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, e, g])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -149,9 +149,9 @@ LogicalProject(c=[$7], g=[$3])
 Calc(select=[c, g])
 +- HashJoin(joinType=[InnerJoin], where=[AND(=(a, d), =(b, e))], select=[d, e, g, a, b, c], isBroadcast=[true], build=[right])
    :- Calc(select=[d, e, g])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -173,10 +173,10 @@ LogicalProject(a=[$0], d=[$3])
 Calc(select=[a, d])
 +- HashJoin(joinType=[InnerJoin], where=[=($f3, d)], select=[a, $f3, d], isBroadcast=[true], build=[right])
    :- Calc(select=[a, +(a, 1) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -197,10 +197,10 @@ LogicalProject(c=[$7], g=[$3])
 Calc(select=[c, g])
 +- HashJoin(joinType=[LeftOuterJoin], where=[AND(=(a, d), <(d, 2))], select=[d, g, a, c], isBroadcast=[true], build=[right])
    :- Calc(select=[d, g])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[a, c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -222,9 +222,9 @@ Calc(select=[c, g])
 +- HashJoin(joinType=[RightOuterJoin], where=[=(b, e)], select=[b, c, e, g], isBroadcast=[true], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[b, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Calc(select=[e, g])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -246,10 +246,10 @@ LogicalProject(c=[$2], g=[$6])
 Calc(select=[c, g])
 +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, c, d, g], isBroadcast=[true], build=[right])
    :- Calc(select=[a, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, g])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -270,10 +270,10 @@ LogicalProject(c=[$2], g=[$6])
 Calc(select=[c, g])
 +- HashJoin(joinType=[LeftOuterJoin], where=[=(b, e)], select=[b, c, e, g], isBroadcast=[true], build=[right])
    :- Calc(select=[b, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, g])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -294,8 +294,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
 HashJoin(joinType=[RightOuterJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[d, e, f, g, h], where=[<(d, 2)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -317,9 +317,9 @@ Calc(select=[c, g])
 +- HashJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[d, g, a, c], isBroadcast=[true], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -348,7 +348,7 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3])
       <![CDATA[
 HashJoin(joinType=[LeftOuterJoin], where=[=(k, k0)], select=[k, v, k0, v0], isBroadcast=[true], build=[right])
 :- Calc(select=[CAST(0:BIGINT) AS k, v], where=[=(k, 0:BIGINT)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
 +- Exchange(distribution=[broadcast])
    +- Values(tuples=[[]], values=[k, v])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
index cbd5d3e..f6fc3f8 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml
@@ -36,17 +36,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))])
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i], where=[<(j, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -69,14 +69,14 @@ LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f, e])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -99,11 +99,11 @@ LogicalFilter(condition=[=($cor1.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -126,14 +126,14 @@ LogicalAggregate(group=[{0}])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(c, f)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f])
       +- HashAggregate(isMerge=[true], groupBy=[e, f], select=[e, f])
          +- Exchange(distribution=[hash[e, f]])
             +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                +- Calc(select=[e, f])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -154,10 +154,10 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -179,14 +179,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, c], build=[right], singleRowJoin=[true])
    :- Calc(select=[a, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[IS NOT NULL(m) AS $f0])
          +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
             +- Exchange(distribution=[single])
                +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                   +- Calc(select=[true AS i])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -211,7 +211,7 @@ LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -221,14 +221,14 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[ri
                   +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[d], build=[right], singleRowJoin=[true])
                      :- Calc(select=[d])
                      :  +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-                     :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                      +- Exchange(distribution=[broadcast])
                         +- Calc(select=[IS NOT NULL(m) AS $f0])
                            +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
                               +- Exchange(distribution=[single])
                                  +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                                     +- Calc(select=[true AS i])
-                                       +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -248,14 +248,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -276,14 +276,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -310,7 +310,7 @@ LogicalUnion(all=[false])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right], singleRowJoin=[true])
 :- Calc(select=[a])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -322,9 +322,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right],
                         +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                            +- Union(all=[true], union=[e, f])
                               :- Calc(select=[e, f], where=[>(d, 10)])
-                              :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                              :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                               +- Calc(select=[j, k], where=[<(i, 100)])
-                                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -350,13 +350,13 @@ LogicalProject(i=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, i), =(b, j))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i, j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -381,13 +381,13 @@ LogicalFilter(condition=[=($cor0.a, $0)])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftSemiJoin], where=[=(b, j)], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[j])
-:        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -411,14 +411,14 @@ LogicalProject(EXPR$0=[$1])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(c, f))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[EXPR$0, f])
       +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
          +- Exchange(distribution=[hash[f]])
             +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                +- Calc(select=[f, e], where=[<(d, 3)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -442,13 +442,13 @@ LogicalProject(EXPR$0=[$4], d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(a, d), =(c, f))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[EXPR$0, d, f])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_COUNT(count1$0) AS EXPR$0])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_COUNT(*) AS count1$0])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -472,10 +472,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)])
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=($f3, EXPR$0), =($f4, e))], select=[a, c, $f3, $f4], isBroadcast=[true], build=[right])
    :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, +(b, 1) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -498,10 +498,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -529,15 +529,15 @@ LogicalProject(e=[$0])
       <![CDATA[
 Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(a, d), =(c, k))], select=[a, b, c], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, d, k])
          +- HashJoin(joinType=[InnerJoin], where=[=(e, j)], select=[e, d, j, k], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[e, d], where=[<(e, 50)])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Calc(select=[j, k], where=[>(i, 10)])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -563,15 +563,15 @@ LogicalProject(j=[$3])
       <![CDATA[
 Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(a, d))], select=[a, b, c], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[j, d])
          +- HashJoin(joinType=[LeftOuterJoin], where=[=(f, k)], select=[f, d, j, k], isBroadcast=[true], build=[right])
             :- Calc(select=[f, d], where=[<(e, 50)])
-            :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[broadcast])
                +- Calc(select=[j, k])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -598,15 +598,15 @@ LogicalProject(d=[$0])
 Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(c, k))], select=[a, c], isBroadcast=[true], build=[right])
    :- Calc(select=[a, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, k])
          +- HashJoin(joinType=[RightOuterJoin], where=[=(d, i)], select=[d, i, k], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[d])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Calc(select=[i, k], where=[>(i, 10)])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -630,11 +630,11 @@ LogicalProject(f1=[$3])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(c, f1), =(a, d))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f0 AS f1, d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -658,10 +658,10 @@ LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)])
 Calc(select=[a, b, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =($f3, EXPR$1), =(b, e))], select=[a, b, c, $f3], isBroadcast=[true], build=[right])
    :- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -683,10 +683,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), >(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -709,10 +709,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -735,13 +735,13 @@ LogicalProject(rk=[$2])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, rk), <>(a, d))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[w0$o0 AS rk, d])
       +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, w0$o0])
          +- Sort(orderBy=[d ASC, e ASC])
             +- Exchange(distribution=[hash[d]])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -774,7 +774,7 @@ Calc(select=[a])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(EXPR$0, i), =(k, c))], select=[a, c, EXPR$0], isBroadcast=[true], build=[right])
    :- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true])
    :  :- Calc(select=[a, c])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :  +- Exchange(distribution=[broadcast])
    :     +- SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :        +- Exchange(distribution=[single])
@@ -782,13 +782,13 @@ Calc(select=[a])
    :              +- Calc(select=[d])
    :                 +- HashJoin(joinType=[InnerJoin], where=[=(e, j)], select=[d, e, j], isBroadcast=[true], build=[right])
    :                    :- Calc(select=[d, e])
-   :                    :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                    :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    :                    +- Exchange(distribution=[broadcast])
    :                       +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])
    :                          +- Exchange(distribution=[hash[j]])
    :                             +- LocalHashAggregate(groupBy=[j], select=[j])
    :                                +- Calc(select=[j])
-   :                                   +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :                                   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[i, k])
          +- Reused(reference_id=[1])
@@ -813,10 +813,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -839,10 +839,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 1)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e], where=[>(d, 10)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -865,10 +865,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[=(a, CAST(b))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -894,14 +894,14 @@ LogicalProject(EXPR$0=[$1])
 Calc(select=[a])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=(b, EXPR$0)], select=[a, b], isBroadcast=[true], build=[right])
    :- Calc(select=[a, b])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[EXPR$0])
          +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
             +- Exchange(distribution=[hash[f]])
                +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                   +- Calc(select=[f, e], where=[<(d, 3)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -923,13 +923,13 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, EXPR$1))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS EXPR$1])
       +- Exchange(distribution=[hash[d]])
          +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -951,10 +951,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -975,10 +975,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1007,16 +1007,16 @@ Calc(select=[a])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=(EXPR$0, j)], select=[a, EXPR$0], isBroadcast=[true], build=[right])
    :- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true])
    :  :- Calc(select=[a])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :  +- Exchange(distribution=[broadcast])
    :     +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :        +- Exchange(distribution=[single])
    :           +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0])
    :              +- Calc(select=[e], where=[>(d, 0)])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[j])
-         +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1048,10 +1048,10 @@ Calc(select=[b])
    :- Calc(select=[b])
    :  +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b], isBroadcast=[true], build=[right])
    :     :- Calc(select=[a, b])
-   :     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[d], where=[>(e, 10)])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[broadcast])
       +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0])
          +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0])
@@ -1059,7 +1059,7 @@ Calc(select=[b])
                +- Exchange(distribution=[single])
                   +- LocalSortAggregate(select=[Partial_SUM(j) AS sum$0])
                      +- Calc(select=[j], where=[<(i, 100)])
-                        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1088,10 +1088,10 @@ Calc(select=[s])
    :     +- Exchange(distribution=[hash[b]])
    :        +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2])
    :           +- Calc(select=[b, a])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1114,11 +1114,11 @@ LogicalProject(f1=[$3])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(c, f1)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f0 AS f1])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1144,10 +1144,10 @@ LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)])
 Calc(select=[a, b, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=($f3, EXPR$0), =($f4, EXPR$1))], select=[a, b, c, $f3, $f4], isBroadcast=[true], build=[right])
    :- Calc(select=[a, b, c, +(a, 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[+(d, 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1171,7 +1171,7 @@ LogicalProject(EXPR$0=[MAX($0) OVER (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOU
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, $0), =(b, $1))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[w0$o0 AS $0, w1$o0 AS $1])
       +- OverAggregate(partitionBy=[f], orderBy=[d ASC], window#0=[MIN(e) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, w0$o0, w1$o0])
@@ -1179,7 +1179,7 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, $0), =(b, $1))], select=[a, b,
             +- Exchange(distribution=[hash[f]])
                +- OverAggregate(window#0=[MAX(d) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[d, e, f, w0$o0])
                   +- Exchange(distribution=[single])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1200,10 +1200,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1226,10 +1226,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d], where=[<(e, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1252,10 +1252,10 @@ LogicalProject(d=[$0])
 Calc(select=[a, b, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=($f3, d)], select=[a, b, c, $f3], isBroadcast=[true], build=[right])
    :- Calc(select=[a, b, c, +(a, 1) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1286,16 +1286,16 @@ LogicalUnion(all=[false])
 Calc(select=[a])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=(b, e)], select=[a, b], isBroadcast=[true], build=[right])
    :- Calc(select=[a, b])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- HashAggregate(isMerge=[true], groupBy=[e], select=[e])
          +- Exchange(distribution=[hash[e]])
             +- LocalHashAggregate(groupBy=[e], select=[e])
                +- Union(all=[true], union=[e])
                   :- Calc(select=[e], where=[>(d, 10)])
-                  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                   +- Calc(select=[CAST(i) AS i], where=[<(i, 100)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1318,18 +1318,18 @@ LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[d], build=[right], singleRowJoin=[true])
       :- Calc(select=[d])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[IS NOT NULL(m) AS $f0])
             +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
                +- Exchange(distribution=[single])
                   +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                      +- Calc(select=[true AS i])
-                        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1359,15 +1359,15 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c, d, e, f],
 :  :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(b, 1)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i], where=[>(i, 50)])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -1404,11 +1404,11 @@ HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e, f], is
 :  :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)])
-:        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i], where=[<(j, 100)])
       +- Reused(reference_id=[1])
@@ -1438,13 +1438,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(a, i))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(c, f))], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d, f])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j, i], where=[<>(k, _UTF-16LE'test':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1470,14 +1470,14 @@ LogicalProject(k=[$2])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
       +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(f, k), =(e, j))], select=[d, e, f], isBroadcast=[true], build=[right])
-         :- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         :- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[k, j])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1503,13 +1503,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(k, c))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j, k])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1534,13 +1534,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(b, j)], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1565,15 +1565,15 @@ LogicalProject(j=[$1])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
       +- HashJoin(joinType=[LeftSemiJoin], where=[=(e, j)], select=[d, e], isBroadcast=[true], build=[right])
          :- Calc(select=[d, e])
-         :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[j])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1603,15 +1603,15 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c, d, e, f],
 :  :- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], isBroadcast=[true], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(b, 1)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i], where=[>(i, 50)])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -1639,10 +1639,10 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1664,14 +1664,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, c], build=[right], singleRowJoin=[true])
    :- Calc(select=[a, c], where=[AND(<=(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[IS NOT NULL(m) AS $f0])
          +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
             +- Exchange(distribution=[single])
                +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                   +- Calc(select=[true AS i])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1691,14 +1691,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1719,14 +1719,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1749,13 +1749,13 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[=(a, c)], select=[a, b], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]], fields=[a, b])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]], fields=[a, b])
 +- Exchange(distribution=[broadcast])
    +- HashAggregate(isMerge=[true], groupBy=[c], select=[c])
       +- Exchange(distribution=[hash[c]])
          +- LocalHashAggregate(groupBy=[c], select=[c])
             +- Calc(select=[c])
-               +- TableSourceScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]], fields=[c, d])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]], fields=[c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -1781,13 +1781,13 @@ LogicalProject(i=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, i), IS NULL(a), IS NULL(i)), =(b, j))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- HashJoin(joinType=[LeftAntiJoin], where=[=(a, d)], select=[a, b, c], isBroadcast=[true], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i, j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1811,10 +1811,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)])
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, EXPR$0), IS NULL($f3), IS NULL(EXPR$0)), =($f4, e))], select=[a, c, $f3, $f4], isBroadcast=[true], build=[right])
    :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, +(b, 1) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1849,13 +1849,13 @@ Calc(select=[b])
    :     :  :- Calc(select=[a, b, c, c0, ck, i0])
    :     :  :  +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], isBroadcast=[true], build=[right])
    :     :  :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])
-   :     :  :     :  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  :     :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  :     :  +- Exchange(distribution=[broadcast])
    :     :  :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
    :     :  :     :        +- Exchange(distribution=[single])
    :     :  :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1])
    :     :  :     :              +- Calc(select=[i], reuse_id=[1])
-   :     :  :     :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2])
+   :     :  :     :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2])
    :     :  :     +- Exchange(distribution=[broadcast])
    :     :  :        +- Calc(select=[i, true AS i0])
    :     :  :           +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])
@@ -1876,7 +1876,7 @@ Calc(select=[b])
    :                    +- Reused(reference_id=[3])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, f])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1898,10 +1898,10 @@ LogicalProject(d=[$0], f=[$2])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), OR(=(c, f), IS NULL(c), IS NULL(f)), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, f, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1923,10 +1923,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1949,10 +1949,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), =(b, e))], select=[a, b, c], isBroadcast=[true], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 1)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e], where=[>(d, 10)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml
index b4fc401..922821b 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.xml
@@ -49,15 +49,15 @@ HashJoin(joinType=[FullOuterJoin], where=[=(a4, a5)], select=[a1, b1, c1, a2, b2
 :     :  :- Exchange(distribution=[hash[a1]])
 :     :  :  +- HashJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2], build=[right])
 :     :  :     :- Exchange(distribution=[hash[a1]])
-:     :  :     :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     :  :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 :     :  :     +- Exchange(distribution=[hash[a2]])
-:     :  :        +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+:     :  :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 :     :  +- Exchange(distribution=[single])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
 :     +- Exchange(distribution=[hash[a4]])
-:        +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 +- Exchange(distribution=[hash[a5]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
 ]]>
     </Resource>
   </TestCase>
@@ -92,15 +92,15 @@ HashJoin(joinType=[LeftOuterJoin], where=[=(a4, a5)], select=[a1, b1, c1, a2, b2
 :  +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a4)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
 :     :- HashJoin(joinType=[LeftOuterJoin], where=[=(a2, a3)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3], isBroadcast=[true], build=[right])
 :     :  :- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2], isBroadcast=[true], build=[right])
-:     :  :  :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+:     :  :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
 :     :  :  +- Exchange(distribution=[broadcast])
-:     :  :     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+:     :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
 :     :  +- Exchange(distribution=[broadcast])
-:     :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+:     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
 :     +- Exchange(distribution=[broadcast])
-:        +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 +- Exchange(distribution=[hash[a5]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
 ]]>
     </Resource>
   </TestCase>
@@ -132,19 +132,19 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3
       <![CDATA[
 Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[LeftOuterJoin], where=[=(a4, a5)], select=[a5, b5, c5, a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    +- Exchange(distribution=[broadcast])
       +- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a4)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[left])
          :- Exchange(distribution=[broadcast])
          :  +- SortMergeJoin(joinType=[RightOuterJoin], where=[=(a2, a3)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3])
          :     :- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2], build=[right])
          :     :  :- Exchange(distribution=[hash[a1]])
-         :     :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+         :     :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
          :     :  +- Exchange(distribution=[hash[a2]])
-         :     :     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
          :     +- Exchange(distribution=[hash[a3]])
-         :        +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
-         +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+         :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -175,17 +175,17 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3
 Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(a2, a4)], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
    :- HashJoin(joinType=[InnerJoin], where=[=(a2, a3)], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :  :- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :  +- Exchange(distribution=[broadcast])
    :     +- HashJoin(joinType=[InnerJoin], where=[=(a5, a3)], select=[a5, b5, c5, a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :        :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        +- Exchange(distribution=[broadcast])
    :           +- HashJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :              :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :              :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :              +- Exchange(distribution=[broadcast])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -217,17 +217,17 @@ Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, c1, a5, b5, c5, a2, b2, c2, a4, b4, c4, a3, b3, c3], isBroadcast=[true], build=[right])
    :- HashJoin(joinType=[InnerJoin], where=[=(b1, b4)], select=[a1, b1, c1, a5, b5, c5, a2, b2, c2, a4, b4, c4], build=[left])
    :  :- Exchange(distribution=[hash[b1]])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :  +- Exchange(distribution=[hash[b4]])
    :     +- HashJoin(joinType=[InnerJoin], where=[=(b5, b4)], select=[a5, b5, c5, a2, b2, c2, a4, b4, c4], isBroadcast=[true], build=[right])
    :        :- HashJoin(joinType=[InnerJoin], where=[=(b5, b2)], select=[a5, b5, c5, a2, b2, c2], isBroadcast=[true], build=[right])
-   :        :  :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        :  +- Exchange(distribution=[broadcast])
-   :        :     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :        :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :        +- Exchange(distribution=[broadcast])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
 ]]>
     </Resource>
   </TestCase>
@@ -262,16 +262,16 @@ Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
    :- NestedLoopJoin(joinType=[FullOuterJoin], where=[=(a2, a3)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3], build=[right])
    :  :- Exchange(distribution=[single])
    :  :  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2], isBroadcast=[true], build=[right])
-   :  :     :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :  :     :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :  :     +- Exchange(distribution=[broadcast])
-   :  :        +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :  :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :  +- Exchange(distribution=[single])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
    +- Exchange(distribution=[broadcast])
       +- HashJoin(joinType=[InnerJoin], where=[=(a4, a5)], select=[a5, b5, c5, a4, b4, c4], isBroadcast=[true], build=[right])
-         :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+         :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
          +- Exchange(distribution=[broadcast])
-            +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -303,18 +303,18 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3
       <![CDATA[
 Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(a4, a5)], select=[a5, b5, c5, a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    +- Exchange(distribution=[broadcast])
       +- HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a4)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
          :- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2, a3, b3, c3], isBroadcast=[true], build=[right])
-         :  :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+         :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
          :  +- Exchange(distribution=[broadcast])
          :     +- HashJoin(joinType=[InnerJoin], where=[=(a2, a3)], select=[a2, b2, c2, a3, b3, c3], isBroadcast=[true], build=[right])
-         :        :- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         :        :- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
          :        +- Exchange(distribution=[broadcast])
-         :           +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+         :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
          +- Exchange(distribution=[broadcast])
-            +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -349,16 +349,16 @@ Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
    :- HashJoin(joinType=[InnerJoin], where=[=(a1, a4)], select=[a1, b1, c1, a2, b2, c2, a5, b5, c5, a4, b4, c4], isBroadcast=[true], build=[right])
    :  :- HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, c1, a2, b2, c2], build=[right])
    :  :  :- Exchange(distribution=[hash[a1]])
-   :  :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :  :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :  :  +- Exchange(distribution=[hash[a2]])
-   :  :     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :  +- Exchange(distribution=[broadcast])
    :     +- HashJoin(joinType=[InnerJoin], where=[=(a4, a5)], select=[a5, b5, c5, a4, b4, c4], isBroadcast=[true], build=[right])
-   :        :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        +- Exchange(distribution=[broadcast])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
 ]]>
     </Resource>
   </TestCase>
@@ -398,18 +398,18 @@ LogicalProject(a4=[$0], b4=[$1], c4=[$2], a1=[$3], b1=[$4], c1=[$5], a2=[$6], b2
 Calc(select=[a4, b4, c4, a1, b1, c1, a2, b2, c2, a3, b3, c3, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[AND(>(+(b2, b4), 100), =(a2, a4))], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
    :- NestedLoopJoin(joinType=[InnerJoin], where=[AND(>(*(b1, b2), 10), =(a2, a3))], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3], build=[right])
-   :  :- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :  +- Exchange(distribution=[broadcast])
    :     +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a5, a3)], select=[a5, b5, c5, a1, b1, c1, a3, b3, c3], build=[right])
    :        :- Calc(select=[a5, b5, c5], where=[<(b5, 15)])
-   :        :  +- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        +- Exchange(distribution=[broadcast])
    :           +- HashJoin(joinType=[InnerJoin], where=[AND(=(a3, a1), <(*(b1, b3), 2000))], select=[a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :              :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :              :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :              +- Exchange(distribution=[broadcast])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -445,7 +445,7 @@ LogicalProject(a3=[$0], b1=[$1], a1=[$2], c2=[$3], c3=[$4], a4=[$5], b4=[$6], a5
       <![CDATA[
 Calc(select=[a3, b1, a1, c2, c3, a4, b4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(a4, a5)], select=[a5, b5, c5, a3, b1, a1, c2, c3, a4, b4], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    +- Exchange(distribution=[broadcast])
       +- HashJoin(joinType=[InnerJoin], where=[=(a1, a4)], select=[a3, b1, a1, c2, c3, a4, b4], isBroadcast=[true], build=[right])
          :- Calc(select=[a3, b1, a1, c2, c3])
@@ -453,16 +453,16 @@ Calc(select=[a3, b1, a1, c2, c3, a4, b4, a5, b5, c5])
          :     :- Calc(select=[b1, a1, a2, c2])
          :     :  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, c2], isBroadcast=[true], build=[right])
          :     :     :- Calc(select=[a1, b1])
-         :     :     :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+         :     :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
          :     :     +- Exchange(distribution=[broadcast])
          :     :        +- Calc(select=[a2, c2])
-         :     :           +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         :     :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
          :     +- Exchange(distribution=[broadcast])
          :        +- Calc(select=[a3, c3])
-         :           +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+         :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[a4, b4])
-               +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -493,17 +493,17 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3
 Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(a2, a4)], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
    :- HashJoin(joinType=[InnerJoin], where=[=(a2, a3)], select=[a2, b2, c2, a5, b5, c5, a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :  :- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :  +- Exchange(distribution=[broadcast])
    :     +- HashJoin(joinType=[InnerJoin], where=[=(a5, a3)], select=[a5, b5, c5, a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :        :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        +- Exchange(distribution=[broadcast])
    :           +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, c1, a3, b3, c3], isBroadcast=[true], build=[right])
-   :              :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :              :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :              +- Exchange(distribution=[broadcast])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
@@ -535,17 +535,17 @@ Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, c1, a5, b5, c5, a2, b2, c2, a4, b4, c4, a3, b3, c3], isBroadcast=[true], build=[right])
    :- HashJoin(joinType=[InnerJoin], where=[=(b1, b4)], select=[a1, b1, c1, a5, b5, c5, a2, b2, c2, a4, b4, c4], build=[left])
    :  :- Exchange(distribution=[hash[b1]])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    :  +- Exchange(distribution=[hash[b4]])
    :     +- HashJoin(joinType=[InnerJoin], where=[=(b5, b4)], select=[a5, b5, c5, a2, b2, c2, a4, b4, c4], isBroadcast=[true], build=[right])
    :        :- HashJoin(joinType=[InnerJoin], where=[=(b5, b2)], select=[a5, b5, c5, a2, b2, c2], isBroadcast=[true], build=[right])
-   :        :  :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+   :        :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
    :        :  +- Exchange(distribution=[broadcast])
-   :        :     +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+   :        :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
    :        +- Exchange(distribution=[broadcast])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
    +- Exchange(distribution=[broadcast])
-      +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
 ]]>
     </Resource>
   </TestCase>
@@ -575,18 +575,18 @@ LogicalProject(a1=[$0], b1=[$1], c1=[$2], a2=[$3], b2=[$4], c2=[$5], a3=[$6], b3
       <![CDATA[
 Calc(select=[a1, b1, c1, a2, b2, c2, a3, b3, c3, a4, b4, c4, a5, b5, c5])
 +- HashJoin(joinType=[InnerJoin], where=[=(c1, c2)], select=[a1, b1, c1, a2, b2, c2, a5, b5, c5, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, T1, source: [TestTableSource(a1, b1, c1)]]], fields=[a1, b1, c1])
    +- Exchange(distribution=[broadcast])
       +- HashJoin(joinType=[InnerJoin], where=[=(c2, c5)], select=[a2, b2, c2, a5, b5, c5, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[left])
          :- Exchange(distribution=[broadcast])
-         :  +- TableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
+         :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(a2, b2, c2)]]], fields=[a2, b2, c2])
          +- HashJoin(joinType=[InnerJoin], where=[=(c3, c4)], select=[a5, b5, c5, a3, b3, c3, a4, b4, c4], isBroadcast=[true], build=[right])
             :- HashJoin(joinType=[InnerJoin], where=[=(c5, c3)], select=[a5, b5, c5, a3, b3, c3], isBroadcast=[true], build=[right])
-            :  :- TableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
+            :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, T5, source: [TestTableSource(a5, b5, c5)]]], fields=[a5, b5, c5])
             :  +- Exchange(distribution=[broadcast])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, T3, source: [TestTableSource(a3, b3, c3)]]], fields=[a3, b3, c3])
             +- Exchange(distribution=[broadcast])
-               +- TableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, T4, source: [TestTableSource(a4, b4, c4)]]], fields=[a4, b4, c4])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.xml
index 6f874ac..0fc9bed 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.xml
@@ -232,7 +232,7 @@ FlinkLogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)], EXPR$2=[SUM($2)], EXPR$3=
       :     +- FlinkLogicalDataStreamTableScan(table=[[default_catalog, default_database, T1]])
       +- FlinkLogicalSnapshot(period=[$cor0.proctime])
          +- FlinkLogicalCalc(select=[id], where=[>(age, 10)])
-            +- FlinkLogicalTableSourceScan(table=[[default_catalog, default_database, temporalTest, source: [TestTemporalTable(id, name, age)]]], fields=[id, name, age])
+            +- FlinkLogicalLegacyTableSourceScan(table=[[default_catalog, default_database, temporalTest, source: [TestTemporalTable(id, name, age)]]], fields=[id, name, age])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
index a4d534c..06e3403 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
@@ -31,9 +31,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -53,9 +53,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[FullOuterJoin], where=[false], select=[d, e, f, g, h, a, b, c], build=[right])
 :- Exchange(distribution=[single])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -75,9 +75,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[FullOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[right])
 :- Exchange(distribution=[single])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -99,10 +99,10 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[FullOuterJoin], where=[AND(=(a, d), <(d, 2))], select=[d, g, a, c], build=[left])
    :- Exchange(distribution=[single])
    :  +- Calc(select=[d, g])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[single])
       +- Calc(select=[a, c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -122,9 +122,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[FullOuterJoin], where=[AND(=(a, d), <(d, 2), <(b, h))], select=[d, e, f, g, h, a, b, c], build=[right])
 :- Exchange(distribution=[single])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -146,10 +146,10 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[FullOuterJoin], where=[=(b, e)], select=[b, c, e, g], build=[left])
    :- Exchange(distribution=[single])
    :  +- Calc(select=[b, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[single])
       +- Calc(select=[e, g])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -169,9 +169,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[FullOuterJoin], where=[<>(a, d)], select=[d, e, f, g, h, a, b, c], build=[right])
 :- Exchange(distribution=[single])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[single])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -196,9 +196,9 @@ LogicalProject(a=[COALESCE($0, $3)], b=[$1], c=[$2], b0=[$4], c0=[$5])
 Calc(select=[CASE(IS NOT NULL(a), a, a0) AS $f0, b, c, b0, c0])
 +- NestedLoopJoin(joinType=[FullOuterJoin], where=[=(a, a0)], select=[a, b, c, a0, b0, c0], build=[left])
    :- Exchange(distribution=[single])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[single])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable3, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -220,9 +220,9 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[d, g, a, c], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -241,9 +241,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftOuterJoin], where=[AND(=(a, d), <(d, 2), <(b, h))], select=[d, e, f, g, h, a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -264,8 +264,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
 NestedLoopJoin(joinType=[InnerJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, f, g, h, a, b, c], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[d, e, f, g, h], where=[<(d, 2)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -288,9 +288,9 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[d, g, a, c], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -312,9 +312,9 @@ LogicalProject(c=[$2], g=[$6])
 Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[AND(=(b, e), =(a, d))], select=[a, b, c, d, e, g], build=[left])
    :- Exchange(distribution=[broadcast])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Calc(select=[d, e, g])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -336,8 +336,8 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[AND(=(a, d), =(b, e))], select=[d, e, g, a, b, c], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, e, g])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -359,10 +359,10 @@ LogicalProject(a=[$0], d=[$3])
 Calc(select=[a, d])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[=($f3, d)], select=[a, $f3, d], build=[right])
    :- Calc(select=[a, +(a, 1) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -383,9 +383,9 @@ LogicalProject(a=[$0], d=[$3])
 NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, d], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[d])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -404,9 +404,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftOuterJoin], where=[<>(a, d)], select=[d, e, f, g, h, a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -425,7 +425,7 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
    +- Values(tuples=[[]], values=[a, b, c])
 ]]>
@@ -446,9 +446,9 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[broadcast])
-   +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -469,10 +469,10 @@ LogicalProject(c=[$7], g=[$3])
 Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[AND(=(a, d), <(d, 2))], select=[d, g, a, c], build=[right])
    :- Calc(select=[d, g])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[a, c])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -494,9 +494,9 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[RightOuterJoin], where=[=(b, e)], select=[b, c, e, g], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[b, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Calc(select=[e, g])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -519,9 +519,9 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, c, d, g], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[a, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Calc(select=[d, g])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -542,7 +542,7 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
 NestedLoopJoin(joinType=[RightOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Values(tuples=[[]], values=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -563,10 +563,10 @@ LogicalProject(c=[$2], g=[$6])
 Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(b, e)], select=[b, c, e, g], build=[right])
    :- Calc(select=[b, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, g])
-         +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -586,8 +586,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[RightOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[left])
 :- Exchange(distribution=[broadcast])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -609,9 +609,9 @@ Calc(select=[c, g])
 +- NestedLoopJoin(joinType=[RightOuterJoin], where=[=(a, d)], select=[d, g, a, c], build=[left])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[<(d, 2)])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Calc(select=[a, c])
-      +- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -631,8 +631,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
       <![CDATA[
 NestedLoopJoin(joinType=[RightOuterJoin], where=[<>(a, d)], select=[d, e, f, g, h, a, b, c], build=[left])
 :- Exchange(distribution=[broadcast])
-:  +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -653,8 +653,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
 NestedLoopJoin(joinType=[RightOuterJoin], where=[AND(=(a, d), <(b, h))], select=[d, e, f, g, h, a, b, c], build=[left])
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[d, e, f, g, h], where=[<(d, 2)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- TableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -683,7 +683,7 @@ LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(k, k0)], select=[k, v, k0, v0], build=[right])
 :- Calc(select=[CAST(0:BIGINT) AS k, v], where=[=(k, 0:BIGINT)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
 +- Exchange(distribution=[broadcast])
    +- Values(tuples=[[]], values=[k, v])
 ]]>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
index 02364c1..2a9023c 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml
@@ -36,17 +36,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i], where=[<(j, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -69,14 +69,14 @@ LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f, e])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -99,11 +99,11 @@ LogicalFilter(condition=[=($cor1.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -124,10 +124,10 @@ LogicalFilter(condition=[<($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[<(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -149,11 +149,11 @@ LogicalFilter(condition=[<($cor0.a, 10)])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[<(a, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- LocalHashAggregate(select=[])
       +- Calc(select=[])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -176,14 +176,14 @@ LogicalAggregate(group=[{0}])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(c, f)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f])
       +- HashAggregate(isMerge=[true], groupBy=[e, f], select=[e, f])
          +- Exchange(distribution=[hash[e, f]])
             +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                +- Calc(select=[e, f])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -204,10 +204,10 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -229,14 +229,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, c], build=[right], singleRowJoin=[true])
    :- Calc(select=[a, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[IS NOT NULL(m) AS $f0])
          +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
             +- Exchange(distribution=[single])
                +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                   +- Calc(select=[true AS i])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -261,7 +261,7 @@ LogicalJoin(condition=[=($1, $3)], joinType=[full])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[c], build=[right], singleRowJoin=[true])
 :- Calc(select=[c])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -271,10 +271,10 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[c], build=[right],
                   +- NestedLoopJoin(joinType=[FullOuterJoin], where=[=(e, j)], select=[e, j], build=[right])
                      :- Exchange(distribution=[single])
                      :  +- Calc(select=[e])
-                     :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                      +- Exchange(distribution=[single])
                         +- Calc(select=[j], where=[>(i, 10)])
-                           +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -299,7 +299,7 @@ LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -309,14 +309,14 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[ri
                   +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[d], build=[right], singleRowJoin=[true])
                      :- Calc(select=[d])
                      :  +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-                     :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                      +- Exchange(distribution=[broadcast])
                         +- Calc(select=[IS NOT NULL(m) AS $f0])
                            +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
                               +- Exchange(distribution=[single])
                                  +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                                     +- Calc(select=[true AS i])
-                                       +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -336,14 +336,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -364,14 +364,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -398,7 +398,7 @@ LogicalUnion(all=[false])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right], singleRowJoin=[true])
 :- Calc(select=[a])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -410,9 +410,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right],
                         +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                            +- Union(all=[true], union=[e, f])
                               :- Calc(select=[e, f], where=[>(d, 10)])
-                              :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                              :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                               +- Calc(select=[j, k], where=[<(i, 100)])
-                                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -438,13 +438,13 @@ LogicalProject(i=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, i), =(b, j))], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i, j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -469,13 +469,13 @@ LogicalFilter(condition=[=($cor0.a, $0)])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(b, j)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[j])
-:        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -511,13 +511,13 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :  :- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, i), IS NULL(b), IS NULL(i)), =(c, k))], select=[a, b, c], build=[right])
 :  :  :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:  :  :  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  :  +- Exchange(distribution=[broadcast])
 :  :  :     +- Calc(select=[d])
-:  :  :        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1])
+:  :  :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1])
 :  :  +- Exchange(distribution=[broadcast])
 :  :     +- Calc(select=[i, k], where=[>(i, 10)])
-:  :        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+:  :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -528,7 +528,7 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :                       +- Exchange(distribution=[hash[l]])
 :                          +- LocalHashAggregate(groupBy=[l], select=[l])
 :                             +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')])
-:                                +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
+:                                +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[e])
       +- Reused(reference_id=[1])
@@ -555,14 +555,14 @@ LogicalProject(EXPR$0=[$1])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(c, f))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[EXPR$0, f])
       +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
          +- Exchange(distribution=[hash[f]])
             +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                +- Calc(select=[f, e], where=[<(d, 3)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -586,13 +586,13 @@ LogicalProject(EXPR$0=[$4], d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(a, d), =(c, f))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[EXPR$0, d, f])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_COUNT(count1$0) AS EXPR$0])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_COUNT(*) AS count1$0])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -616,10 +616,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)])
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=($f3, EXPR$0), =($f4, e))], select=[a, c, $f3, $f4], build=[right])
    :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, +(b, 1) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -642,10 +642,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -674,13 +674,13 @@ Calc(select=[b])
    :- Calc(select=[b, c, CASE(AND(<>(c0, 0), IS NOT NULL(i), IS NOT NULL(a)), 1, 2) AS $f3])
    :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, 1)], select=[a, b, c, c0, i], build=[right])
    :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])
-   :     :  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  +- Exchange(distribution=[broadcast])
    :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
    :     :        +- Exchange(distribution=[single])
    :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
    :     :              +- Calc(select=[1 AS EXPR$0], reuse_id=[1])
-   :     :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+   :     :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[true AS i])
    :           +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -689,7 +689,7 @@ Calc(select=[b])
    :                    +- Reused(reference_id=[1])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, f])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -717,15 +717,15 @@ LogicalProject(e=[$0])
       <![CDATA[
 Calc(select=[c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(a, d), =(c, k))], select=[a, b, c], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, d, k])
          +- NestedLoopJoin(joinType=[InnerJoin], where=[=(e, j)], select=[e, d, j, k], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[e, d], where=[<(e, 50)])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Calc(select=[j, k], where=[>(i, 10)])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -751,15 +751,15 @@ LogicalProject(j=[$3])
       <![CDATA[
 Calc(select=[c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(a, d))], select=[a, b, c], build=[right])
-   :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[j, d])
          +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(f, k)], select=[f, d, j, k], build=[right])
             :- Calc(select=[f, d], where=[<(e, 50)])
-            :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[broadcast])
                +- Calc(select=[j, k])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -786,15 +786,15 @@ LogicalProject(d=[$0])
 Calc(select=[c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(c, k))], select=[a, c], build=[right])
    :- Calc(select=[a, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, k])
          +- NestedLoopJoin(joinType=[RightOuterJoin], where=[=(d, i)], select=[d, i, k], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[d])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Calc(select=[i, k], where=[>(i, 10)])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -818,11 +818,11 @@ LogicalProject(f1=[$3])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(c, f1), =(a, d))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f0 AS f1, d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -846,10 +846,10 @@ LogicalProject(d=[$0], EXPR$1=[SUBSTRING($2, 1, 5)])
 Calc(select=[a, b, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =($f3, EXPR$1), =(b, e))], select=[a, b, c, $f3], build=[right])
    :- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -871,10 +871,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), >(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -897,10 +897,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -923,13 +923,13 @@ LogicalProject(rk=[$2])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, rk), <>(a, d))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[w0$o0 AS rk, d])
       +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, w0$o0])
          +- Sort(orderBy=[d ASC, e ASC])
             +- Exchange(distribution=[hash[d]])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -962,7 +962,7 @@ Calc(select=[a])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(EXPR$0, i), =(k, c))], select=[a, c, EXPR$0], build=[right])
    :- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true])
    :  :- Calc(select=[a, c])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :  +- Exchange(distribution=[broadcast])
    :     +- SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :        +- Exchange(distribution=[single])
@@ -970,13 +970,13 @@ Calc(select=[a])
    :              +- Calc(select=[d])
    :                 +- NestedLoopJoin(joinType=[InnerJoin], where=[=(e, j)], select=[d, e, j], build=[right])
    :                    :- Calc(select=[d, e])
-   :                    :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                    :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    :                    +- Exchange(distribution=[broadcast])
    :                       +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])
    :                          +- Exchange(distribution=[hash[j]])
    :                             +- LocalHashAggregate(groupBy=[j], select=[j])
    :                                +- Calc(select=[j])
-   :                                   +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :                                   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[i, k])
          +- Reused(reference_id=[1])
@@ -1001,10 +1001,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1027,10 +1027,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 1)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e], where=[>(d, 10)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1053,10 +1053,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[=(a, CAST(b))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1082,14 +1082,14 @@ LogicalProject(EXPR$0=[$1])
 Calc(select=[a])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(b, EXPR$0)], select=[a, b], build=[right])
    :- Calc(select=[a, b])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[EXPR$0])
          +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
             +- Exchange(distribution=[hash[f]])
                +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                   +- Calc(select=[f, e], where=[<(d, 3)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1111,13 +1111,13 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, EXPR$1))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS EXPR$1])
       +- Exchange(distribution=[hash[d]])
          +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1139,10 +1139,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1163,10 +1163,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1198,13 +1198,13 @@ Calc(select=[b])
    :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, 1)], select=[a, b, c, i], build=[right])
    :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])
    :     :  :- Calc(select=[a, b])
-   :     :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  +- Exchange(distribution=[broadcast])
    :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
    :     :        +- Exchange(distribution=[single])
    :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
    :     :              +- Calc(select=[1 AS EXPR$0], reuse_id=[1])
-   :     :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+   :     :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[true AS i])
    :           +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -1213,7 +1213,7 @@ Calc(select=[b])
    :                    +- Reused(reference_id=[1])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1242,16 +1242,16 @@ Calc(select=[a])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(EXPR$0, j)], select=[a, EXPR$0], build=[right])
    :- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true])
    :  :- Calc(select=[a])
-   :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :  +- Exchange(distribution=[broadcast])
    :     +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :        +- Exchange(distribution=[single])
    :           +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0])
    :              +- Calc(select=[e], where=[>(d, 0)])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[j])
-         +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1283,10 +1283,10 @@ Calc(select=[b])
    :- Calc(select=[b])
    :  +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b], build=[right])
    :     :- Calc(select=[a, b])
-   :     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[d], where=[>(e, 10)])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[broadcast])
       +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0])
          +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0])
@@ -1294,7 +1294,7 @@ Calc(select=[b])
                +- Exchange(distribution=[single])
                   +- LocalSortAggregate(select=[Partial_SUM(j) AS sum$0])
                      +- Calc(select=[j], where=[<(i, 100)])
-                        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1323,10 +1323,10 @@ Calc(select=[s])
    :     +- Exchange(distribution=[hash[b]])
    :        +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2])
    :           +- Calc(select=[b, a])
-   :              +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1356,16 +1356,16 @@ LogicalProject(d=[$0])
 Calc(select=[c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, c], build=[right])
    :- Calc(select=[a, c])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
          +- NestedLoopJoin(joinType=[FullOuterJoin], where=[=(e, i0)], select=[d, e, i0], build=[right])
             :- Exchange(distribution=[single])
             :  +- Calc(select=[d, e])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[single])
                +- Calc(select=[CAST(i) AS i0], where=[>(i, 10)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1388,11 +1388,11 @@ LogicalProject(f1=[$3])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(c, f1)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[f0 AS f1])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1418,10 +1418,10 @@ LogicalProject(EXPR$0=[+($0, 100)], EXPR$1=[SUBSTRING($2, 1, 5)])
 Calc(select=[a, b, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=($f3, EXPR$0), =($f4, EXPR$1))], select=[a, b, c, $f3, $f4], build=[right])
    :- Calc(select=[a, b, c, +(a, 10) AS $f3, SUBSTRING(c, 1, 5) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[+(d, 100) AS EXPR$0, SUBSTRING(f, 1, 5) AS EXPR$1])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1445,7 +1445,7 @@ LogicalProject(EXPR$0=[MAX($0) OVER (RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOU
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, $0), =(b, $1))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[w0$o0 AS $0, w1$o0 AS $1])
       +- OverAggregate(partitionBy=[f], orderBy=[d ASC], window#0=[MIN(e) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, w0$o0, w1$o0])
@@ -1453,7 +1453,7 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, $0), =(b, $1))], select=
             +- Exchange(distribution=[hash[f]])
                +- OverAggregate(window#0=[MAX(d) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[d, e, f, w0$o0])
                   +- Exchange(distribution=[single])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1474,10 +1474,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1500,10 +1500,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d], where=[<(e, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1526,10 +1526,10 @@ LogicalProject(d=[$0])
 Calc(select=[a, b, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=($f3, d)], select=[a, b, c, $f3], build=[right])
    :- Calc(select=[a, b, c, +(a, 1) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1560,16 +1560,16 @@ LogicalUnion(all=[false])
 Calc(select=[a])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(b, e)], select=[a, b], build=[right])
    :- Calc(select=[a, b])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- HashAggregate(isMerge=[true], groupBy=[e], select=[e])
          +- Exchange(distribution=[hash[e]])
             +- LocalHashAggregate(groupBy=[e], select=[e])
                +- Union(all=[true], union=[e])
                   :- Calc(select=[e], where=[>(d, 10)])
-                  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                   +- Calc(select=[CAST(i) AS i], where=[<(i, 100)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1594,17 +1594,17 @@ LogicalFilter(condition=[<($1, 100)])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[<>(a, d)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i], where=[<(j, 100)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1627,18 +1627,18 @@ LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[d], build=[right], singleRowJoin=[true])
       :- Calc(select=[d])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[IS NOT NULL(m) AS $f0])
             +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
                +- Exchange(distribution=[single])
                   +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                      +- Calc(select=[true AS i])
-                        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1668,15 +1668,15 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c, d, e, f],
 :  :- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(b, 1)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i], where=[>(i, 50)])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -1710,13 +1710,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(a, i))], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(c, f))], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d, f])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j, i], where=[<>(k, _UTF-16LE'test':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1742,14 +1742,14 @@ LogicalProject(k=[$2])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
       +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(f, k), =(e, j))], select=[d, e, f], build=[right])
-         :- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         :- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[k, j])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1775,13 +1775,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(k, c))], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j, k])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1806,13 +1806,13 @@ LogicalProject(j=[$1])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(b, j)], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1837,15 +1837,15 @@ LogicalProject(j=[$1])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
       +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[=(e, j)], select=[d, e], build=[right])
          :- Calc(select=[d, e])
-         :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[j])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1875,11 +1875,11 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c, d, e,
 :  :- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)])
-:        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i], where=[<(j, 100)])
       +- Reused(reference_id=[1])
@@ -1912,15 +1912,15 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c, d, e, f],
 :  :- NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f], build=[left])
 :  :  :- Exchange(distribution=[broadcast])
 :  :  :  +- Calc(select=[a, b, c], where=[>=(b, 1)])
-:  :  :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-:  :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:  :  :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :           +- Exchange(distribution=[single])
 :              +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                 +- Calc(select=[true AS i], where=[>(i, 50)])
-:                    +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+:                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -1953,15 +1953,15 @@ LogicalProject(j=[$1])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=(a, d), IS NULL(a), IS NULL(d))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
       +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(e, j), IS NULL(e), IS NULL(j)), =(i, d))], select=[d, e], build=[right])
          :- Calc(select=[d, e])
-         :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
          +- Exchange(distribution=[broadcast])
             +- Calc(select=[j, i])
-               +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1982,10 +1982,10 @@ LogicalFilter(condition=[<($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[<(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2008,11 +2008,11 @@ LogicalFilter(condition=[<($cor0.a, 10)])
 Calc(select=[a, b, c])
 +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f3], select=[a, b, c, $f3], build=[right])
    :- Calc(select=[a, b, c, <(a, 10) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- LocalHashAggregate(select=[])
          +- Calc(select=[])
-            +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2033,10 +2033,10 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2058,14 +2058,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, c], build=[right], singleRowJoin=[true])
    :- Calc(select=[a, c], where=[AND(<=(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[IS NOT NULL(m) AS $f0])
          +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
             +- Exchange(distribution=[single])
                +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                   +- Calc(select=[true AS i])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2085,14 +2085,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2113,14 +2113,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2143,13 +2143,13 @@ LogicalFilter(condition=[=($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, c)], select=[a, b], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]], fields=[a, b])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]], fields=[a, b])
 +- Exchange(distribution=[broadcast])
    +- HashAggregate(isMerge=[true], groupBy=[c], select=[c])
       +- Exchange(distribution=[hash[c]])
          +- LocalHashAggregate(groupBy=[c], select=[c])
             +- Calc(select=[c])
-               +- TableSourceScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]], fields=[c, d])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]], fields=[c, d])
 ]]>
     </Resource>
   </TestCase>
@@ -2175,13 +2175,13 @@ LogicalProject(i=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, i), IS NULL(a), IS NULL(i)), =(b, j))], select=[a, b, c], build=[right])
 :- NestedLoopJoin(joinType=[LeftAntiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
-:  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[d])
-:        +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[i, j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -2205,10 +2205,10 @@ LogicalProject(EXPR$0=[SUBSTRING($2, 1, 5)])
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=($f3, EXPR$0), IS NULL($f3), IS NULL(EXPR$0)), =($f4, e))], select=[a, c, $f3, $f4], build=[right])
    :- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, +(b, 1) AS $f4])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2243,13 +2243,13 @@ Calc(select=[b])
    :     :  :- Calc(select=[a, b, c, c0, ck, i0])
    :     :  :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], build=[right])
    :     :  :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])
-   :     :  :     :  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  :     :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  :     :  +- Exchange(distribution=[broadcast])
    :     :  :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
    :     :  :     :        +- Exchange(distribution=[single])
    :     :  :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1])
    :     :  :     :              +- Calc(select=[i], reuse_id=[1])
-   :     :  :     :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2])
+   :     :  :     :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[2])
    :     :  :     +- Exchange(distribution=[broadcast])
    :     :  :        +- Calc(select=[i, true AS i0])
    :     :  :           +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])
@@ -2270,7 +2270,7 @@ Calc(select=[b])
    :                    +- Reused(reference_id=[3])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d, f])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2292,10 +2292,10 @@ LogicalProject(d=[$0], f=[$2])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), OR(=(c, f), IS NULL(c), IS NULL(f)), =(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, f, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2317,10 +2317,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), >(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2342,10 +2342,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), >(b, 10))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2367,10 +2367,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), =(b, e))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2393,10 +2393,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), =(b, e))], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 1)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, e], where=[>(d, 10)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2417,10 +2417,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=(a, d), IS NULL(a), IS NULL(d))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2442,10 +2442,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=(a, d), IS NULL(a), IS NULL(d))], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[AND(>(b, 10), LIKE(c, _UTF-16LE'abc'))])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2480,12 +2480,12 @@ Calc(select=[b])
    :     :  :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, ck, i, i0], build=[right])
    :     :  :     :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])
    :     :  :     :  :- Calc(select=[a, b])
-   :     :  :     :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  :     :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     :  :     :  +- Exchange(distribution=[broadcast])
    :     :  :     :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
    :     :  :     :        +- Exchange(distribution=[single])
    :     :  :     :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1])
-   :     :  :     :              +- TableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]], fields=[i], reuse_id=[1])
+   :     :  :     :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, t1, source: [TestTableSource(i)]]], fields=[i], reuse_id=[1])
    :     :  :     +- Exchange(distribution=[broadcast])
    :     :  :        +- Calc(select=[i, true AS i0])
    :     :  :           +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])
@@ -2496,7 +2496,7 @@ Calc(select=[b])
    :     :     +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck])
    :     :        +- Exchange(distribution=[single])
    :     :           +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1])
-   :     :              +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]], fields=[j], reuse_id=[2])
+   :     :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(j)]]], fields=[j], reuse_id=[2])
    :     +- Exchange(distribution=[broadcast])
    :        +- Calc(select=[j, true AS i])
    :           +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])
@@ -2505,7 +2505,7 @@ Calc(select=[b])
    :                    +- Reused(reference_id=[2])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2526,10 +2526,10 @@ LogicalProject(d=[$0], f=[$2])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(a, d), IS NULL(a), IS NULL(d)), OR(=(c, f), IS NULL(c), IS NULL(f)))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d, f])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2550,10 +2550,10 @@ LogicalProject(d=[$0])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=(a, d), IS NULL(a), IS NULL(d))], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2576,10 +2576,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=(a, d), IS NULL(a), IS NULL(d))], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d], where=[<(e, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -2602,10 +2602,10 @@ LogicalProject(d=[$0])
 Calc(select=[a, b, c])
 +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(=($f3, d), IS NULL($f3), IS NULL(d))], select=[a, b, c, $f3], build=[right])
    :- Calc(select=[a, b, c, *(a, b) AS $f3])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml
index 52e98cc..51c923c 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml
@@ -37,17 +37,17 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))])
 HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
 :  +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:     :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[broadcast])
 :        +- Calc(select=[IS NOT NULL(m) AS $f0])
 :           +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
 :              +- Exchange(distribution=[single])
 :                 +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
 :                    +- Calc(select=[true AS i])
-:                       +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[i]])
    +- Calc(select=[i], where=[<(j, 100)])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -71,14 +71,14 @@ LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$0=[MAX($4)])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(c, f))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[b, c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[e, f]])
    +- Calc(select=[f, e])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f])
                +- Calc(select=[d, e, f], where=[<(d, 100)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -102,11 +102,11 @@ LogicalFilter(condition=[=($cor1.a, $0)])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -127,10 +127,10 @@ LogicalFilter(condition=[<($cor0.a, $0)])
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[<(a, d)], select=[a, b, c], build=[right])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -152,11 +152,11 @@ LogicalFilter(condition=[<($cor0.a, 10)])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[true], select=[a, b, c], build=[right])
 :- Calc(select=[a, b, c], where=[<(a, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- LocalHashAggregate(select=[])
       +- Calc(select=[])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -180,14 +180,14 @@ LogicalAggregate(group=[{0}])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(c, f)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f]])
    +- Calc(select=[f])
       +- HashAggregate(isMerge=[true], groupBy=[e, f], select=[e, f])
          +- Exchange(distribution=[hash[e, f]])
             +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                +- Calc(select=[e, f])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -209,10 +209,10 @@ LogicalFilter(condition=[=($cor0.a, $0)])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -234,14 +234,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
 Calc(select=[+(a, 10) AS EXPR$0, c])
 +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, c], build=[right], singleRowJoin=[true])
    :- Calc(select=[a, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[IS NOT NULL(m) AS $f0])
          +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
             +- Exchange(distribution=[single])
                +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                   +- Calc(select=[true AS i])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -266,7 +266,7 @@ LogicalJoin(condition=[=($1, $3)], joinType=[full])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[c], build=[right], singleRowJoin=[true])
 :- Calc(select=[c])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -276,10 +276,10 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[c], build=[right],
                   +- HashJoin(joinType=[FullOuterJoin], where=[=(e, j)], select=[e, j], build=[right])
                      :- Exchange(distribution=[hash[e]])
                      :  +- Calc(select=[e])
-                     :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                      +- Exchange(distribution=[hash[j]])
                         +- Calc(select=[j], where=[>(i, 10)])
-                           +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                           +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -304,7 +304,7 @@ LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -314,14 +314,14 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[ri
                   +- NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[d], build=[right], singleRowJoin=[true])
                      :- Calc(select=[d])
                      :  +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-                     :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                      +- Exchange(distribution=[broadcast])
                         +- Calc(select=[IS NOT NULL(m) AS $f0])
                            +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
                               +- Exchange(distribution=[single])
                                  +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                                     +- Calc(select=[true AS i])
-                                       +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -341,14 +341,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
     <Resource name="planAfter">
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
-:- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -369,14 +369,14 @@ LogicalTableScan(table=[[default_catalog, default_database, r, source: [TestTabl
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a, b, c], build=[right], singleRowJoin=[true])
 :- Calc(select=[a, b, c], where=[>(b, 10)])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
          +- Exchange(distribution=[single])
             +- LocalHashAggregate(select=[Partial_MIN(i) AS min$0])
                +- Calc(select=[true AS i])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -403,7 +403,7 @@ LogicalUnion(all=[false])
       <![CDATA[
 NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right], singleRowJoin=[true])
 :- Calc(select=[a])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[IS NOT NULL(m) AS $f0])
       +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -415,9 +415,9 @@ NestedLoopJoin(joinType=[LeftSemiJoin], where=[$f0], select=[a], build=[right],
                         +- LocalHashAggregate(groupBy=[e, f], select=[e, f])
                            +- Union(all=[true], union=[e, f])
                               :- Calc(select=[e, f], where=[>(d, 10)])
-                              :  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                              :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
                               +- Calc(select=[j, k], where=[<(i, 100)])
-                                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -445,13 +445,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, i), =(b, j))], select=[a, b, c
 :- Exchange(distribution=[hash[a, b]])
 :  +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :     :- Exchange(distribution=[hash[a]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
 :        +- Calc(select=[d])
-:           +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[i, j]])
    +- Calc(select=[i, j])
-      +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -478,13 +478,13 @@ HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[righ
 :- Exchange(distribution=[hash[a]])
 :  +- HashJoin(joinType=[LeftSemiJoin], where=[=(b, j)], select=[a, b, c], build=[right])
 :     :- Exchange(distribution=[hash[b]])
-:     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[j]])
 :        +- Calc(select=[j])
-:           +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -522,13 +522,13 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :  :  :- Exchange(distribution=[hash[c]])
 :  :  :  +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :  :  :     :- Exchange(distribution=[hash[a]])
-:  :  :     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  :  :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :  :  :     +- Exchange(distribution=[hash[d]])
 :  :  :        +- Calc(select=[d])
-:  :  :           +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1])
+:  :  :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], reuse_id=[1])
 :  :  +- Exchange(distribution=[hash[k]])
 :  :     +- Calc(select=[i, k], where=[>(i, 10)])
-:  :        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+:  :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 :  +- Exchange(distribution=[broadcast])
 :     +- Calc(select=[IS NOT NULL(m) AS $f0])
 :        +- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS m])
@@ -539,7 +539,7 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[<>(b, e)], select=[a, b, c], buil
 :                       +- Exchange(distribution=[hash[l]])
 :                          +- LocalHashAggregate(groupBy=[l], select=[l])
 :                             +- Calc(select=[l], where=[LIKE(n, _UTF-16LE'Test')])
-:                                +- TableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
+:                                +- LegacyTableSourceScan(table=[[default_catalog, default_database, t2, source: [TestTableSource(l, m, n)]]], fields=[l, m, n])
 +- Exchange(distribution=[broadcast])
    +- Calc(select=[e])
       +- Reused(reference_id=[1])
@@ -567,14 +567,14 @@ LogicalProject(EXPR$0=[$1])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(c, f))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[b, c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[EXPR$0, f]])
    +- Calc(select=[EXPR$0, f])
       +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
          +- Exchange(distribution=[hash[f]])
             +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                +- Calc(select=[f, e], where=[<(d, 3)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -599,13 +599,13 @@ LogicalProject(EXPR$0=[$4], d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, EXPR$0), =(a, d), =(c, f))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[b, a, c]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[EXPR$0, d, f]])
    +- Calc(select=[EXPR$0, d, f])
       +- HashAggregate(isMerge=[true], groupBy=[d, e, f], select=[d, e, f, Final_COUNT(count1$0) AS EXPR$0])
          +- Exchange(distribution=[hash[d, e, f]])
             +- LocalHashAggregate(groupBy=[d, e, f], select=[d, e, f, Partial_COUNT(*) AS count1$0])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -630,10 +630,10 @@ Calc(select=[+(a, 10) AS EXPR$0, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=($f3, EXPR$0), =($f4, e))], select=[a, c, $f3, $f4], build=[right])
    :- Exchange(distribution=[hash[$f3, $f4]])
    :  +- Calc(select=[a, c, SUBSTRING(c, 1, 5) AS $f3, +(b, 1) AS $f4])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[EXPR$0, e]])
       +- Calc(select=[SUBSTRING(f, 1, 5) AS EXPR$0, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -657,10 +657,10 @@ LogicalProject(d=[$0])
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[left])
 :- Exchange(distribution=[hash[a, b]])
 :  +- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d, e]])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -690,13 +690,13 @@ Calc(select=[b])
    :  +- Calc(select=[b, c, CASE(AND(<>(c0, 0), IS NOT NULL(i), IS NOT NULL(a)), 1, 2) AS $f3])
    :     +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, 1)], select=[a, b, c, c0, i], build=[right])
    :        :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])
-   :        :  :- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :        :  :- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :  +- Exchange(distribution=[broadcast])
    :        :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
    :        :        +- Exchange(distribution=[single])
    :        :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
    :        :              +- Calc(select=[1 AS EXPR$0], reuse_id=[1])
-   :        :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+   :        :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
    :        +- Exchange(distribution=[broadcast])
    :           +- Calc(select=[true AS i])
    :              +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -705,7 +705,7 @@ Calc(select=[b])
    :                       +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[d, f]])
       +- Calc(select=[d, f])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -734,16 +734,16 @@ LogicalProject(e=[$0])
 Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, e), =(a, d), =(c, k))], select=[a, b, c], build=[right])
    :- Exchange(distribution=[hash[b, a, c]])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[e, d, k]])
       +- Calc(select=[e, d, k])
          +- HashJoin(joinType=[InnerJoin], where=[=(e, j)], select=[e, d, j, k], build=[left])
             :- Exchange(distribution=[hash[e]])
             :  +- Calc(select=[e, d], where=[<(e, 50)])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[hash[j]])
                +- Calc(select=[j, k], where=[>(i, 10)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -770,16 +770,16 @@ LogicalProject(j=[$3])
 Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, j), =(a, d))], select=[a, b, c], build=[right])
    :- Exchange(distribution=[hash[b, a]])
-   :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[j, d]])
       +- Calc(select=[j, d])
          +- HashJoin(joinType=[LeftOuterJoin], where=[=(f, k)], select=[f, d, j, k], build=[left])
             :- Exchange(distribution=[hash[f]])
             :  +- Calc(select=[f, d], where=[<(e, 50)])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[hash[k]])
                +- Calc(select=[j, k])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -807,16 +807,16 @@ Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(c, k))], select=[a, c], build=[right])
    :- Exchange(distribution=[hash[a, c]])
    :  +- Calc(select=[a, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d, k]])
       +- Calc(select=[d, k])
          +- HashJoin(joinType=[RightOuterJoin], where=[=(d, i)], select=[d, i, k], build=[left])
             :- Exchange(distribution=[hash[d]])
             :  +- Calc(select=[d])
-            :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+            :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
             +- Exchange(distribution=[hash[i]])
                +- Calc(select=[i, k], where=[>(i, 10)])
-                  +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -841,11 +841,11 @@ LogicalProject(f1=[$3])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(c, f1), =(a, d))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[c, a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[f1, d]])
    +- Calc(select=[f0 AS f1, d])
       +- Correlate(invocation=[table_func($cor0.f)], correlate=[table(table_func($cor0.f))], select=[d,e,f,f0], rowType=[RecordType(INTEGER d, BIGINT e, VARCHAR(2147483647) f, VARCHAR(2147483647) f0)], joinType=[INNER])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -870,10 +870,10 @@ Calc(select=[a, b, c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =($f3, EXPR$1), =(b, e))], select=[a, b, c, $f3], build=[right])
    :- Exchange(distribution=[hash[a, $f3, b]])
    :  +- Calc(select=[a, b, c, SUBSTRING(c, 1, 5) AS $f3])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d, EXPR$1, e]])
       +- Calc(select=[d, SUBSTRING(f, 1, 5) AS EXPR$1, e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -896,10 +896,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), >(b, e))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -923,10 +923,10 @@ LogicalProject(d=[$0])
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
 :  +- Calc(select=[a, b, c], where=[>(b, 10)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -950,13 +950,13 @@ LogicalProject(rk=[$2])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(b, rk), <>(a, d))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[b]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[rk]])
    +- Calc(select=[w0$o0 AS rk, d])
       +- OverAggregate(partitionBy=[d], orderBy=[e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[d, e, f, w0$o0])
          +- Sort(orderBy=[d ASC, e ASC])
             +- Exchange(distribution=[hash[d]])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -990,7 +990,7 @@ Calc(select=[a])
    :- Exchange(distribution=[hash[EXPR$0, c]])
    :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, c, EXPR$0], build=[right], singleRowJoin=[true])
    :     :- Calc(select=[a, c])
-   :     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Exchange(distribution=[broadcast])
    :        +- SortAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :           +- Exchange(distribution=[single])
@@ -999,12 +999,12 @@ Calc(select=[a])
    :                    +- HashJoin(joinType=[InnerJoin], where=[=(e, j)], select=[d, e, j], build=[right])
    :                       :- Exchange(distribution=[hash[e]])
    :                       :  +- Calc(select=[d, e])
-   :                       :     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                       :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    :                       +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])
    :                          +- Exchange(distribution=[hash[j]])
    :                             +- LocalHashAggregate(groupBy=[j], select=[j])
    :                                +- Calc(select=[j])
-   :                                   +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
+   :                                   +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1])
    +- Exchange(distribution=[hash[i, k]])
       +- Calc(select=[i, k])
          +- Reused(reference_id=[1])
@@ -1030,10 +1030,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a, b]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d, e]])
    +- Calc(select=[d, e])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1057,10 +1057,10 @@ LogicalProject(d=[$0])
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, e))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a, b]])
 :  +- Calc(select=[a, b, c], where=[>(b, 1)])
-:     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d, e]])
    +- Calc(select=[d, e], where=[>(d, 10)])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1084,10 +1084,10 @@ LogicalProject(d=[$0])
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[left])
 :- Exchange(distribution=[hash[a]])
 :  +- Calc(select=[a, b, c], where=[=(a, CAST(b))])
-:     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1114,14 +1114,14 @@ Calc(select=[a])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=(b, EXPR$0)], select=[a, b], build=[right])
    :- Exchange(distribution=[hash[b]])
    :  +- Calc(select=[a, b])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[EXPR$0]])
       +- Calc(select=[EXPR$0])
          +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_MAX(max$0) AS EXPR$0])
             +- Exchange(distribution=[hash[f]])
                +- LocalHashAggregate(groupBy=[f], select=[f, Partial_MAX(e) AS max$0])
                   +- Calc(select=[f, e], where=[<(d, 3)])
-                     +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+                     +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1144,13 +1144,13 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT()])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[AND(=(a, d), =(b, EXPR$1))], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a, b]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d, EXPR$1]])
    +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count1$0) AS EXPR$1])
       +- Exchange(distribution=[hash[d]])
          +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(*) AS count1$0])
             +- Calc(select=[d])
-               +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1173,10 +1173,10 @@ LogicalProject(d=[$0])
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
 :  +- Calc(select=[a, b, c], where=[AND(>(b, 10), NOT(LIKE(c, _UTF-16LE'abc')))])
-:     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1198,10 +1198,10 @@ LogicalProject(d=[$0])
       <![CDATA[
 HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b, c], build=[right])
 :- Exchange(distribution=[hash[a]])
-:  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[d]])
    +- Calc(select=[d])
-      +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1234,13 +1234,13 @@ Calc(select=[b])
    :     +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a, 1)], select=[a, b, c, i], build=[right])
    :        :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])
    :        :  :- Calc(select=[a, b])
-   :        :  :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :        :  :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :        :  +- Exchange(distribution=[broadcast])
    :        :     +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c])
    :        :        +- Exchange(distribution=[single])
    :        :           +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0])
    :        :              +- Calc(select=[1 AS EXPR$0], reuse_id=[1])
-   :        :                 +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+   :        :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
    :        +- Exchange(distribution=[broadcast])
    :           +- Calc(select=[true AS i])
    :              +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])
@@ -1249,7 +1249,7 @@ Calc(select=[b])
    :                       +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1279,16 +1279,16 @@ Calc(select=[a])
    :- Exchange(distribution=[hash[EXPR$0]])
    :  +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[true], select=[a, EXPR$0], build=[right], singleRowJoin=[true])
    :     :- Calc(select=[a])
-   :     :  +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Exchange(distribution=[broadcast])
    :        +- HashAggregate(isMerge=[true], select=[Final_MAX(max$0) AS EXPR$0])
    :           +- Exchange(distribution=[single])
    :              +- LocalHashAggregate(select=[Partial_MAX(e) AS max$0])
    :                 +- Calc(select=[e], where=[>(d, 0)])
-   :                    +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :                    +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[hash[j]])
       +- Calc(select=[j])
-         +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1321,10 +1321,10 @@ Calc(select=[b])
    :  +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, b], build=[right])
    :     :- Exchange(distribution=[hash[a]])
    :     :  +- Calc(select=[a, b])
-   :     :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    :     +- Exchange(distribution=[hash[d]])
    :        +- Calc(select=[d], where=[>(e, 10)])
-   :           +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
    +- Exchange(distribution=[broadcast])
       +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0])
          +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0])
@@ -1332,7 +1332,7 @@ Calc(select=[b])
                +- Exchange(distribution=[single])
                   +- LocalSortAggregate(select=[Partial_SUM(j) AS sum$0])
                      +- Calc(select=[j], where=[<(i, 100)])
-                        +- TableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
+                        +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k])
 ]]>
     </Resource>
   </TestCase>
@@ -1362,10 +1362,10 @@ Calc(select=[s])
    :        +- Exchange(distribution=[hash[b]])
    :           +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(*) AS count1$1, Partial_MAX(b) AS max$2])
    :              +- Calc(select=[b, a])
-   :                 +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[e]])
       +- Calc(select=[e])
-         +- TableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -1396,16 +1396,16 @@ Calc(select=[c])
 +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, d)], select=[a, c], build=[right])
    :- Exchange(distribution=[hash[a]])
    :  +- Calc(select=[a, c])
-   :     +- TableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d])
          +- HashJoin(joinType=[FullOuterJoin], where=[=(e, i0)], select=[d, e, i0], build=[right])
             :- Exchange(distribution=[hash[e]])
             :  +- Calc(select=[d, e])
... 11931 lines suppressed ...