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 2019/07/24 05:13:38 UTC

[flink] branch master updated (869ccd6 -> c601cfd)

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

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


    from 869ccd6  [FLINK-13345][tests] Dump jstack output for Flink JVMs
     new 66e55d6  [FLINK-13266][table] Move OptimizerConfigOptions & ExecutionConfigOptions to table-api-java module
     new 9a6ca54  [FLINK-13266][table] Relocate blink runtime classes to avoid class clashes
     new c601cfd  [FLINK-13266][table] Relocate blink planner classes to avoid class clashes

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../flink/table/api/EnvironmentSettings.java       |    4 +-
 .../table/api/config/ExecutionConfigOptions.java   |  207 ++
 .../table/api/config/OptimizerConfigOptions.java   |   90 +
 .../apache/calcite/sql2rel/RelDecorrelator.java    |    3 +-
 .../flink/table/api/OptimizerConfigOptions.java    |   88 -
 .../table/calcite/FlinkCalciteCatalogReader.java   |   78 -
 .../flink/table/calcite/type/FlinkReturnTypes.java |  154 -
 .../type/NumericExceptFirstOperandChecker.java     |   92 -
 .../type/NumericOrDefaultReturnTypeInference.java  |   68 -
 .../type/RepeatFamilyOperandTypeChecker.java       |  125 -
 .../flink/table/catalog/CatalogCalciteSchema.java  |   91 -
 .../table/catalog/CatalogManagerCalciteSchema.java |   91 -
 .../flink/table/catalog/DatabaseCalciteSchema.java |  199 --
 .../apache/flink/table/catalog/FlinkSchema.java    |   59 -
 .../catalog/FunctionCatalogOperatorTable.java      |  142 -
 .../catalog/QueryOperationCatalogViewTable.java    |  106 -
 .../apache/flink/table/executor/BatchExecutor.java |  110 -
 .../flink/table/executor/BlinkExecutorFactory.java |   81 -
 .../apache/flink/table/executor/ExecutorBase.java  |   72 -
 .../flink/table/executor/StreamExecutor.java       |   56 -
 .../flink/table/expressions/ExpressionBuilder.java |  143 -
 .../expressions/PlannerTypeInferenceUtilImpl.java  |  144 -
 .../expressions/ResolvedAggInputReference.java     |   96 -
 .../expressions/ResolvedAggLocalReference.java     |   98 -
 .../expressions/ResolvedDistinctKeyReference.java  |   88 -
 .../flink/table/expressions/RexNodeConverter.java  | 1015 -------
 .../flink/table/expressions/RexNodeExpression.java |   70 -
 .../table/expressions/SqlAggFunctionVisitor.java   |  100 -
 .../functions/InternalFunctionDefinitions.java     |   37 -
 .../functions/aggfunctions/AvgAggFunction.java     |  181 --
 .../functions/aggfunctions/CollectAggFunction.java |  131 -
 .../functions/aggfunctions/ConcatAggFunction.java  |  115 -
 .../aggfunctions/ConcatWithRetractAggFunction.java |  138 -
 .../ConcatWsWithRetractAggFunction.java            |  149 -
 .../functions/aggfunctions/Count1AggFunction.java  |   91 -
 .../functions/aggfunctions/CountAggFunction.java   |   92 -
 .../aggfunctions/DeclarativeAggregateFunction.java |  172 --
 .../aggfunctions/DenseRankAggFunction.java         |   83 -
 .../aggfunctions/FirstValueAggFunction.java        |  221 --
 .../FirstValueWithRetractAggFunction.java          |  408 ---
 .../functions/aggfunctions/IncrSumAggFunction.java |  172 --
 .../IncrSumWithRetractAggFunction.java             |  225 --
 .../aggfunctions/LastValueAggFunction.java         |  220 --
 .../LastValueWithRetractAggFunction.java           |  407 ---
 .../functions/aggfunctions/LeadLagAggFunction.java |  291 --
 .../functions/aggfunctions/MaxAggFunction.java     |  220 --
 .../aggfunctions/MaxWithRetractAggFunction.java    |  405 ---
 .../functions/aggfunctions/MinAggFunction.java     |  220 --
 .../aggfunctions/MinWithRetractAggFunction.java    |  405 ---
 .../functions/aggfunctions/RankAggFunction.java    |   96 -
 .../aggfunctions/RankLikeAggFunctionBase.java      |  131 -
 .../aggfunctions/RowNumberAggFunction.java         |   85 -
 .../aggfunctions/SingleValueAggFunction.java       |  282 --
 .../functions/aggfunctions/Sum0AggFunction.java    |  210 --
 .../functions/aggfunctions/SumAggFunction.java     |  168 --
 .../aggfunctions/SumWithRetractAggFunction.java    |  222 --
 .../table/functions/sql/CalciteSqlFunction.java    |   69 -
 .../table/functions/sql/FlinkSqlOperatorTable.java | 1196 --------
 .../sql/ProctimeMaterializeSqlFunction.java        |   66 -
 .../table/functions/sql/SqlConcatAggFunction.java  |   66 -
 .../sql/SqlFirstLastValueAggFunction.java          |   65 -
 .../table/functions/sql/SqlIncrSumAggFunction.java |   75 -
 .../functions/sql/SqlThrowExceptionFunction.java   |   41 -
 .../sql/StreamRecordTimestampSqlFunction.java      |   55 -
 .../sql/internal/SqlAuxiliaryGroupAggFunction.java |   47 -
 .../functions/tablefunctions/ReplicateRows.java    |   71 -
 .../table/operations/DataStreamQueryOperation.java |  140 -
 .../table/operations/PlannerQueryOperation.java    |   84 -
 .../operations/RichTableSourceQueryOperation.java  |   74 -
 .../apache/flink/table/plan/PartialFinalType.java  |   41 -
 .../flink/table/plan/QueryOperationConverter.java  |  552 ----
 .../flink/table/plan/metadata/FlinkMetadata.java   |  243 --
 .../table/plan/metadata/FlinkRelMetadataQuery.java |  250 --
 .../table/plan/nodes/exec/ExecNodeVisitor.java     |   33 -
 .../table/plan/nodes/exec/ExecNodeVisitorImpl.java |   33 -
 .../plan/nodes/process/DAGProcessContext.java      |   42 -
 .../table/plan/nodes/process/DAGProcessor.java     |   34 -
 .../table/plan/nodes/resource/NodeResource.java    |   57 -
 .../plan/nodes/resource/NodeResourceUtil.java      |   77 -
 .../parallelism/FinalParallelismSetter.java        |  135 -
 .../resource/parallelism/ParallelismProcessor.java |   73 -
 .../nodes/resource/parallelism/ShuffleStage.java   |   88 -
 .../parallelism/ShuffleStageGenerator.java         |  157 -
 .../ShuffleStageParallelismCalculator.java         |   92 -
 ...FlinkAggregateExpandDistinctAggregatesRule.java |  925 ------
 .../logical/FlinkAggregateJoinTransposeRule.java   |  554 ----
 .../rules/logical/FlinkAggregateRemoveRule.java    |  131 -
 .../plan/rules/logical/FlinkFilterJoinRule.java    |  363 ---
 .../logical/FlinkJoinPushExpressionsRule.java      |   83 -
 .../logical/FlinkProjectJoinTransposeRule.java     |  151 -
 .../FlinkSemiAntiJoinFilterTransposeRule.java      |   91 -
 .../FlinkSemiAntiJoinJoinTransposeRule.java        |  281 --
 .../FlinkSemiAntiJoinProjectTransposeRule.java     |  202 --
 .../plan/rules/logical/SubQueryDecorrelator.java   | 1440 ---------
 .../apache/flink/table/plan/util/JoinTypeUtil.java |   52 -
 .../flink/table/plan/util/KeySelectorUtil.java     |   68 -
 .../apache/flink/table/plan/util/OperatorType.java |   40 -
 .../flink/table/planner/BlinkPlannerFactory.java   |   72 -
 .../apache/flink/table/planner/PlannerContext.java |  236 --
 .../planner/catalog/CatalogCalciteSchema.java      |   92 +
 .../catalog/CatalogManagerCalciteSchema.java       |   94 +
 .../planner/catalog/DatabaseCalciteSchema.java     |  205 ++
 .../flink/table/planner/catalog/FlinkSchema.java   |   59 +
 .../catalog/FunctionCatalogOperatorTable.java      |  144 +
 .../catalog/QueryOperationCatalogViewTable.java    |  107 +
 .../table/planner/delegation/BatchExecutor.java    |  110 +
 .../planner/delegation/BlinkExecutorFactory.java   |   81 +
 .../planner/delegation/BlinkPlannerFactory.java    |   72 +
 .../table/planner/delegation/ExecutorBase.java     |   72 +
 .../table/planner/delegation/PlannerContext.java   |  236 ++
 .../table/planner/delegation/StreamExecutor.java   |   56 +
 .../planner/expressions/ExpressionBuilder.java     |  145 +
 .../expressions/PlannerTypeInferenceUtilImpl.java  |  147 +
 .../expressions/ResolvedAggInputReference.java     |   99 +
 .../expressions/ResolvedAggLocalReference.java     |  100 +
 .../expressions/ResolvedDistinctKeyReference.java  |   90 +
 .../planner/expressions/RexNodeConverter.java      | 1029 +++++++
 .../planner/expressions/RexNodeExpression.java     |   73 +
 .../planner/expressions/SqlAggFunctionVisitor.java |  103 +
 .../functions/InternalFunctionDefinitions.java     |   38 +
 .../functions/aggfunctions/AvgAggFunction.java     |  181 ++
 .../functions/aggfunctions/CollectAggFunction.java |  131 +
 .../functions/aggfunctions/ConcatAggFunction.java  |  115 +
 .../aggfunctions/ConcatWithRetractAggFunction.java |  138 +
 .../ConcatWsWithRetractAggFunction.java            |  149 +
 .../functions/aggfunctions/Count1AggFunction.java  |   91 +
 .../functions/aggfunctions/CountAggFunction.java   |   92 +
 .../aggfunctions/DeclarativeAggregateFunction.java |  172 ++
 .../aggfunctions/DenseRankAggFunction.java         |   83 +
 .../aggfunctions/FirstValueAggFunction.java        |  221 ++
 .../FirstValueWithRetractAggFunction.java          |  408 +++
 .../functions/aggfunctions/IncrSumAggFunction.java |  172 ++
 .../IncrSumWithRetractAggFunction.java             |  225 ++
 .../aggfunctions/LastValueAggFunction.java         |  220 ++
 .../LastValueWithRetractAggFunction.java           |  407 +++
 .../functions/aggfunctions/LeadLagAggFunction.java |  291 ++
 .../functions/aggfunctions/MaxAggFunction.java     |  220 ++
 .../aggfunctions/MaxWithRetractAggFunction.java    |  405 +++
 .../functions/aggfunctions/MinAggFunction.java     |  220 ++
 .../aggfunctions/MinWithRetractAggFunction.java    |  405 +++
 .../functions/aggfunctions/RankAggFunction.java    |   96 +
 .../aggfunctions/RankLikeAggFunctionBase.java      |  131 +
 .../aggfunctions/RowNumberAggFunction.java         |   85 +
 .../aggfunctions/SingleValueAggFunction.java       |  282 ++
 .../functions/aggfunctions/Sum0AggFunction.java    |  210 ++
 .../functions/aggfunctions/SumAggFunction.java     |  168 ++
 .../aggfunctions/SumWithRetractAggFunction.java    |  222 ++
 .../planner/functions/sql/CalciteSqlFunction.java  |   69 +
 .../functions/sql/FlinkSqlOperatorTable.java       | 1196 ++++++++
 .../sql/ProctimeMaterializeSqlFunction.java        |   66 +
 .../functions/sql/SqlConcatAggFunction.java        |   66 +
 .../sql/SqlFirstLastValueAggFunction.java          |   65 +
 .../functions/sql/SqlIncrSumAggFunction.java       |   75 +
 .../functions/sql/SqlThrowExceptionFunction.java   |   41 +
 .../sql/StreamRecordTimestampSqlFunction.java      |   55 +
 .../sql/internal/SqlAuxiliaryGroupAggFunction.java |   47 +
 .../functions/tablefunctions/ReplicateRows.java    |   71 +
 .../operations/DataStreamQueryOperation.java       |  144 +
 .../planner/operations/PlannerQueryOperation.java  |   88 +
 .../operations/RichTableSourceQueryOperation.java  |   77 +
 .../planner/operations/SqlConversionException.java |   35 +
 .../operations/SqlToOperationConverter.java        |  215 ++
 .../planner/plan/FlinkCalciteCatalogReader.java    |   78 +
 .../flink/table/planner/plan/PartialFinalType.java |   41 +
 .../planner/plan/QueryOperationConverter.java      |  552 ++++
 .../table/planner/plan/metadata/FlinkMetadata.java |  243 ++
 .../plan/metadata/FlinkRelMetadataQuery.java       |  250 ++
 .../planner/plan/nodes/exec/ExecNodeVisitor.java   |   33 +
 .../plan/nodes/exec/ExecNodeVisitorImpl.java       |   33 +
 .../plan/nodes/process/DAGProcessContext.java      |   42 +
 .../planner/plan/nodes/process/DAGProcessor.java   |   34 +
 .../planner/plan/nodes/resource/NodeResource.java  |   57 +
 .../plan/nodes/resource/NodeResourceUtil.java      |   77 +
 .../parallelism/FinalParallelismSetter.java        |  135 +
 .../resource/parallelism/ParallelismProcessor.java |   73 +
 .../nodes/resource/parallelism/ShuffleStage.java   |   88 +
 .../parallelism/ShuffleStageGenerator.java         |  157 +
 .../ShuffleStageParallelismCalculator.java         |   92 +
 ...FlinkAggregateExpandDistinctAggregatesRule.java |  925 ++++++
 .../logical/FlinkAggregateJoinTransposeRule.java   |  554 ++++
 .../rules/logical/FlinkAggregateRemoveRule.java    |  131 +
 .../plan/rules/logical/FlinkFilterJoinRule.java    |  363 +++
 .../logical/FlinkJoinPushExpressionsRule.java      |   83 +
 .../logical/FlinkProjectJoinTransposeRule.java     |  151 +
 .../FlinkSemiAntiJoinFilterTransposeRule.java      |   91 +
 .../FlinkSemiAntiJoinJoinTransposeRule.java        |  281 ++
 .../FlinkSemiAntiJoinProjectTransposeRule.java     |  202 ++
 .../plan/rules/logical/SubQueryDecorrelator.java   | 1440 +++++++++
 .../table/planner/plan/type/FlinkReturnTypes.java  |  154 +
 .../type/NumericExceptFirstOperandChecker.java     |   92 +
 .../type/NumericOrDefaultReturnTypeInference.java  |   68 +
 .../plan/type/RepeatFamilyOperandTypeChecker.java  |  125 +
 .../table/planner/plan/utils/JoinTypeUtil.java     |   52 +
 .../table/planner/plan/utils/KeySelectorUtil.java  |   68 +
 .../table/planner/plan/utils/OperatorType.java     |   40 +
 .../planner/utils/AggregatePhaseStrategy.java      |   43 +
 .../table/planner/utils/SingleElementIterator.java |   69 +
 .../table/planner/utils/TableConfigUtils.java      |  118 +
 .../table/sqlexec/SqlConversionException.java      |   35 -
 .../table/sqlexec/SqlToOperationConverter.java     |  216 --
 .../flink/table/util/AggregatePhaseStrategy.java   |   43 -
 .../flink/table/util/SingleElementIterator.java    |   69 -
 .../apache/flink/table/util/TableConfigUtils.java  |  118 -
 .../org.apache.flink.table.factories.TableFactory  |    4 +-
 .../apache/flink/table/calcite/CalciteConfig.scala |  228 --
 .../table/calcite/FlinkCalciteSqlValidator.scala   |   95 -
 .../apache/flink/table/calcite/FlinkContext.scala  |   45 -
 .../flink/table/calcite/FlinkContextImpl.scala     |   32 -
 .../apache/flink/table/calcite/FlinkLocalRef.scala |   48 -
 .../table/calcite/FlinkLogicalRelFactories.scala   |  262 --
 .../flink/table/calcite/FlinkPlannerImpl.scala     |  210 --
 .../flink/table/calcite/FlinkRelBuilder.scala      |  169 --
 .../flink/table/calcite/FlinkRelFactories.scala    |  124 -
 .../flink/table/calcite/FlinkRelOptCluster.scala   |   77 -
 .../table/calcite/FlinkRelOptClusterFactory.scala  |   43 -
 .../flink/table/calcite/FlinkTypeFactory.scala     |  518 ----
 .../flink/table/calcite/FlinkTypeSystem.scala      |  148 -
 .../flink/table/calcite/PreValidateReWriter.scala  |  201 --
 .../table/calcite/RelTimeIndicatorConverter.scala  |  700 -----
 .../flink/table/codegen/CalcCodeGenerator.scala    |  214 --
 .../flink/table/codegen/CodeGenException.scala     |   24 -
 .../apache/flink/table/codegen/CodeGenUtils.scala  |  720 -----
 .../flink/table/codegen/CodeGeneratorContext.scala |  830 -----
 .../table/codegen/CollectorCodeGenerator.scala     |   98 -
 .../table/codegen/CorrelateCodeGenerator.scala     |  426 ---
 .../table/codegen/EqualiserCodeGenerator.scala     |  147 -
 .../flink/table/codegen/ExpandCodeGenerator.scala  |   77 -
 .../flink/table/codegen/ExprCodeGenerator.scala    |  755 -----
 .../flink/table/codegen/ExpressionReducer.scala    |  255 --
 .../table/codegen/FunctionCodeGenerator.scala      |  218 --
 .../apache/flink/table/codegen/GenerateUtils.scala |  826 -----
 .../flink/table/codegen/GeneratedExpression.scala  |  103 -
 .../flink/table/codegen/HashCodeGenerator.scala    |  106 -
 .../org/apache/flink/table/codegen/Indenter.scala  |   58 -
 .../table/codegen/InputFormatCodeGenerator.scala   |   93 -
 .../table/codegen/LongHashJoinGenerator.scala      |  361 ---
 .../table/codegen/LookupJoinCodeGenerator.scala    |  456 ---
 .../flink/table/codegen/MatchCodeGenerator.scala   |  814 -----
 .../codegen/NestedLoopJoinCodeGenerator.scala      |  370 ---
 .../table/codegen/OperatorCodeGenerator.scala      |  232 --
 .../table/codegen/ProjectionCodeGenerator.scala    |  214 --
 .../flink/table/codegen/SinkCodeGenerator.scala    |  274 --
 .../flink/table/codegen/ValuesCodeGenerator.scala  |   60 -
 .../flink/table/codegen/agg/AggCodeGen.scala       |   49 -
 .../codegen/agg/AggsHandlerCodeGenerator.scala     |  825 -----
 .../table/codegen/agg/DeclarativeAggCodeGen.scala  |  308 --
 .../table/codegen/agg/DistinctAggCodeGen.scala     |  934 ------
 .../table/codegen/agg/ImperativeAggCodeGen.scala   |  505 ----
 .../table/codegen/agg/batch/AggCodeGenHelper.scala |  714 -----
 .../agg/batch/AggWithoutKeysCodeGenerator.scala    |  118 -
 .../codegen/agg/batch/HashAggCodeGenHelper.scala   |  845 ------
 .../codegen/agg/batch/HashAggCodeGenerator.scala   |  230 --
 .../agg/batch/HashWindowCodeGenerator.scala        |  815 -----
 .../codegen/agg/batch/SortAggCodeGenerator.scala   |  149 -
 .../agg/batch/SortWindowCodeGenerator.scala        |  269 --
 .../codegen/agg/batch/WindowCodeGenerator.scala    |  764 -----
 .../flink/table/codegen/calls/BuiltInMethods.scala |  493 ---
 .../flink/table/codegen/calls/CallGenerator.scala  |   35 -
 .../table/codegen/calls/ConstantCallGen.scala      |   37 -
 .../codegen/calls/CurrentTimePointCallGen.scala    |   58 -
 .../flink/table/codegen/calls/DivCallGen.scala     |   67 -
 .../flink/table/codegen/calls/ExtractCallGen.scala |  115 -
 .../table/codegen/calls/FloorCeilCallGen.scala     |   93 -
 .../table/codegen/calls/FunctionGenerator.scala    |  923 ------
 .../table/codegen/calls/HashCodeCallGen.scala      |   36 -
 .../flink/table/codegen/calls/IfCallGen.scala      |   67 -
 .../flink/table/codegen/calls/LikeCallGen.scala    |  171 --
 .../flink/table/codegen/calls/MethodCallGen.scala  |   70 -
 .../flink/table/codegen/calls/PrintCallGen.scala   |   67 -
 .../flink/table/codegen/calls/RandCallGen.scala    |   56 -
 .../codegen/calls/ScalarFunctionCallGen.scala      |  138 -
 .../table/codegen/calls/ScalarOperatorGens.scala   | 2176 -------------
 .../flink/table/codegen/calls/StringCallGen.scala  |  814 -----
 .../table/codegen/calls/TableFunctionCallGen.scala |   93 -
 .../table/codegen/calls/TimestampDiffCallGen.scala |  120 -
 ...ltiFieldRangeBoundComparatorCodeGenerator.scala |   84 -
 .../over/RangeBoundComparatorCodeGenerator.scala   |  177 --
 .../codegen/sort/ComparatorCodeGenerator.scala     |   84 -
 .../table/codegen/sort/SortCodeGenerator.scala     |  464 ---
 .../apache/flink/table/dataview/DataViewSpec.scala |   47 -
 .../flink/table/dataview/DataViewUtils.scala       |  214 --
 .../flink/table/expressions/ExpressionBridge.scala |   40 -
 .../flink/table/expressions/InputTypeSpec.scala    |   69 -
 .../table/expressions/PlannerExpression.scala      |   87 -
 .../expressions/PlannerExpressionConverter.scala   |  848 ------
 .../expressions/PlannerExpressionParserImpl.scala  |    8 +-
 .../table/expressions/PlannerExpressionUtils.scala |   68 -
 .../flink/table/expressions/Reinterpret.scala      |   45 -
 .../table/expressions/ReturnTypeInference.scala    |  217 --
 .../flink/table/expressions/aggregations.scala     |  230 --
 .../flink/table/expressions/arithmetic.scala       |  149 -
 .../org/apache/flink/table/expressions/call.scala  |  223 --
 .../org/apache/flink/table/expressions/cast.scala  |   44 -
 .../flink/table/expressions/collection.scala       |  174 --
 .../flink/table/expressions/comparison.scala       |  181 --
 .../apache/flink/table/expressions/composite.scala |  100 -
 .../flink/table/expressions/fieldExpression.scala  |  230 --
 .../flink/table/expressions/hashExpressions.scala  |   93 -
 .../apache/flink/table/expressions/literals.scala  |   82 -
 .../org/apache/flink/table/expressions/logic.scala |   87 -
 .../flink/table/expressions/mathExpressions.scala  |  401 ---
 .../apache/flink/table/expressions/ordering.scala  |   43 -
 .../flink/table/expressions/overOffsets.scala      |   54 -
 .../expressions/plannerWindowProperties.scala      |   86 -
 .../table/expressions/stringExpressions.scala      |  490 ---
 .../apache/flink/table/expressions/subquery.scala  |   76 -
 .../apache/flink/table/expressions/symbols.scala   |  127 -
 .../org/apache/flink/table/expressions/time.scala  |  322 --
 .../flink/table/expressions/windowProperties.scala |   61 -
 .../table/functions/utils/AggSqlFunction.scala     |  216 --
 .../table/functions/utils/ScalarSqlFunction.scala  |  199 --
 .../table/functions/utils/TableSqlFunction.scala   |  191 --
 .../functions/utils/UserDefinedFunctionUtils.scala |  858 ------
 .../scala/org/apache/flink/table/package.scala     |   43 -
 .../org/apache/flink/table/plan/TreeNode.scala     |  115 -
 .../apache/flink/table/plan/cost/FlinkCost.scala   |  354 ---
 .../flink/table/plan/cost/FlinkCostBase.scala      |   37 -
 .../flink/table/plan/cost/FlinkCostFactory.scala   |   49 -
 .../table/plan/cost/FlinkCostFactoryBase.scala     |   35 -
 .../flink/table/plan/logical/MatchRecognize.scala  |   42 -
 .../flink/table/plan/logical/groupWindows.scala    |   78 -
 .../metadata/AggCallSelectivityEstimator.scala     |  455 ---
 .../metadata/FlinkDefaultRelMetadataProvider.scala |   53 -
 .../plan/metadata/FlinkRelMdColumnInterval.scala   |  746 -----
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |  295 --
 .../metadata/FlinkRelMdColumnOriginNullCount.scala |  134 -
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |  739 -----
 .../plan/metadata/FlinkRelMdCumulativeCost.scala   |   59 -
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |  626 ----
 .../plan/metadata/FlinkRelMdDistribution.scala     |  103 -
 .../FlinkRelMdFilteredColumnInterval.scala         |  264 --
 .../metadata/FlinkRelMdModifiedMonotonicity.scala  |  552 ----
 .../metadata/FlinkRelMdNonCumulativeCost.scala     |   55 -
 .../FlinkRelMdPercentageOriginalRows.scala         |  146 -
 .../plan/metadata/FlinkRelMdPopulationSize.scala   |  412 ---
 .../table/plan/metadata/FlinkRelMdRowCount.scala   |  457 ---
 .../plan/metadata/FlinkRelMdSelectivity.scala      |  301 --
 .../flink/table/plan/metadata/FlinkRelMdSize.scala |  436 ---
 .../plan/metadata/FlinkRelMdUniqueGroups.scala     |  432 ---
 .../table/plan/metadata/FlinkRelMdUniqueKeys.scala |  566 ----
 .../table/plan/metadata/SelectivityEstimator.scala | 1146 -------
 .../flink/table/plan/nodes/FlinkConventions.scala  |   55 -
 .../flink/table/plan/nodes/FlinkRelNode.scala      |  118 -
 .../flink/table/plan/nodes/calcite/Expand.scala    |  109 -
 .../table/plan/nodes/calcite/LogicalExpand.scala   |   58 -
 .../table/plan/nodes/calcite/LogicalRank.scala     |   96 -
 .../table/plan/nodes/calcite/LogicalSink.scala     |   57 -
 .../nodes/calcite/LogicalWatermarkAssigner.scala   |   43 -
 .../nodes/calcite/LogicalWindowAggregate.scala     |   89 -
 .../flink/table/plan/nodes/calcite/Rank.scala      |  137 -
 .../flink/table/plan/nodes/calcite/Sink.scala      |   58 -
 .../plan/nodes/calcite/WatermarkAssigner.scala     |   65 -
 .../table/plan/nodes/calcite/WindowAggregate.scala |   87 -
 .../flink/table/plan/nodes/common/CommonCalc.scala |   86 -
 .../nodes/common/CommonIntermediateTableScan.scala |   54 -
 .../table/plan/nodes/common/CommonLookupJoin.scala |  677 -----
 .../plan/nodes/common/CommonPhysicalExchange.scala |  109 -
 .../plan/nodes/common/CommonPhysicalJoin.scala     |   86 -
 .../table/plan/nodes/exec/BatchExecNode.scala      |   35 -
 .../flink/table/plan/nodes/exec/ExecNode.scala     |   97 -
 .../table/plan/nodes/exec/StreamExecNode.scala     |   27 -
 .../plan/nodes/logical/FlinkLogicalAggregate.scala |  162 -
 .../plan/nodes/logical/FlinkLogicalCalc.scala      |   79 -
 .../plan/nodes/logical/FlinkLogicalCorrelate.scala |   99 -
 .../logical/FlinkLogicalDataStreamTableScan.scala  |  100 -
 .../plan/nodes/logical/FlinkLogicalExpand.scala    |   82 -
 .../FlinkLogicalIntermediateTableScan.scala        |   92 -
 .../plan/nodes/logical/FlinkLogicalIntersect.scala |   89 -
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |  108 -
 .../plan/nodes/logical/FlinkLogicalMatch.scala     |  132 -
 .../plan/nodes/logical/FlinkLogicalMinus.scala     |   87 -
 .../nodes/logical/FlinkLogicalOverAggregate.scala  |  106 -
 .../plan/nodes/logical/FlinkLogicalRank.scala      |  123 -
 .../table/plan/nodes/logical/FlinkLogicalRel.scala |   28 -
 .../plan/nodes/logical/FlinkLogicalSink.scala      |   80 -
 .../plan/nodes/logical/FlinkLogicalSnapshot.scala  |   90 -
 .../plan/nodes/logical/FlinkLogicalSort.scala      |  139 -
 .../logical/FlinkLogicalTableFunctionScan.scala    |  125 -
 .../logical/FlinkLogicalTableSourceScan.scala      |  124 -
 .../plan/nodes/logical/FlinkLogicalUnion.scala     |   93 -
 .../plan/nodes/logical/FlinkLogicalValues.scala    |   89 -
 .../logical/FlinkLogicalWatermarkAssigner.scala    |   79 -
 .../logical/FlinkLogicalWindowAggregate.scala      |  119 -
 .../plan/nodes/physical/FlinkPhysicalRel.scala     |   42 -
 .../nodes/physical/PhysicalTableSourceScan.scala   |   55 -
 .../batch/BatchExecBoundedStreamScan.scala         |  118 -
 .../plan/nodes/physical/batch/BatchExecCalc.scala  |  166 -
 .../nodes/physical/batch/BatchExecCorrelate.scala  |  207 --
 .../nodes/physical/batch/BatchExecExchange.scala   |  196 --
 .../nodes/physical/batch/BatchExecExpand.scala     |  111 -
 .../batch/BatchExecGroupAggregateBase.scala        |   94 -
 .../physical/batch/BatchExecHashAggregate.scala    |  155 -
 .../batch/BatchExecHashAggregateBase.scala         |  157 -
 .../nodes/physical/batch/BatchExecHashJoin.scala   |  285 --
 .../batch/BatchExecHashWindowAggregate.scala       |   98 -
 .../batch/BatchExecHashWindowAggregateBase.scala   |  159 -
 .../batch/BatchExecIntermediateTableScan.scala     |   44 -
 .../nodes/physical/batch/BatchExecJoinBase.scala   |  225 --
 .../plan/nodes/physical/batch/BatchExecLimit.scala |  121 -
 .../batch/BatchExecLocalHashAggregate.scala        |  132 -
 .../batch/BatchExecLocalHashWindowAggregate.scala  |   93 -
 .../batch/BatchExecLocalSortAggregate.scala        |  141 -
 .../batch/BatchExecLocalSortWindowAggregate.scala  |   91 -
 .../nodes/physical/batch/BatchExecLookupJoin.scala |  100 -
 .../physical/batch/BatchExecNestedLoopJoin.scala   |  184 --
 .../physical/batch/BatchExecOverAggregate.scala    |  630 ----
 .../plan/nodes/physical/batch/BatchExecRank.scala  |  304 --
 .../plan/nodes/physical/batch/BatchExecSink.scala  |  189 --
 .../plan/nodes/physical/batch/BatchExecSort.scala  |  138 -
 .../physical/batch/BatchExecSortAggregate.scala    |  170 --
 .../batch/BatchExecSortAggregateBase.scala         |  130 -
 .../nodes/physical/batch/BatchExecSortLimit.scala  |  158 -
 .../physical/batch/BatchExecSortMergeJoin.scala    |  287 --
 .../batch/BatchExecSortWindowAggregate.scala       |   98 -
 .../batch/BatchExecSortWindowAggregateBase.scala   |  145 -
 .../physical/batch/BatchExecTableSourceScan.scala  |  159 -
 .../plan/nodes/physical/batch/BatchExecUnion.scala |  119 -
 .../nodes/physical/batch/BatchExecValues.scala     |   88 -
 .../batch/BatchExecWindowAggregateBase.scala       |   83 -
 .../nodes/physical/batch/BatchPhysicalRel.scala    |   29 -
 .../nodes/physical/stream/StreamExecCalc.scala     |  132 -
 .../physical/stream/StreamExecCorrelate.scala      |  143 -
 .../physical/stream/StreamExecDataStreamScan.scala |  176 --
 .../physical/stream/StreamExecDeduplicate.scala    |  165 -
 .../nodes/physical/stream/StreamExecExchange.scala |  114 -
 .../nodes/physical/stream/StreamExecExpand.scala   |  110 -
 .../stream/StreamExecGlobalGroupAggregate.scala    |  226 --
 .../physical/stream/StreamExecGroupAggregate.scala |  220 --
 .../stream/StreamExecGroupAggregateBase.scala      |   54 -
 .../stream/StreamExecGroupWindowAggregate.scala    |  366 ---
 .../StreamExecIncrementalGroupAggregate.scala      |  221 --
 .../stream/StreamExecIntermediateTableScan.scala   |   61 -
 .../nodes/physical/stream/StreamExecJoin.scala     |  258 --
 .../nodes/physical/stream/StreamExecLimit.scala    |  183 --
 .../stream/StreamExecLocalGroupAggregate.scala     |  160 -
 .../physical/stream/StreamExecLookupJoin.scala     |  110 -
 .../nodes/physical/stream/StreamExecMatch.scala    |  463 ---
 .../physical/stream/StreamExecOverAggregate.scala  |  469 ---
 .../nodes/physical/stream/StreamExecRank.scala     |  254 --
 .../nodes/physical/stream/StreamExecSink.scala     |  239 --
 .../nodes/physical/stream/StreamExecSort.scala     |  154 -
 .../physical/stream/StreamExecSortLimit.scala      |  238 --
 .../stream/StreamExecTableSourceScan.scala         |  254 --
 .../physical/stream/StreamExecTemporalJoin.scala   |  426 ---
 .../physical/stream/StreamExecTemporalSort.scala   |  204 --
 .../nodes/physical/stream/StreamExecUnion.scala    |   90 -
 .../nodes/physical/stream/StreamExecValues.scala   |   92 -
 .../stream/StreamExecWatermarkAssigner.scala       |  184 --
 .../physical/stream/StreamExecWindowJoin.scala     |  360 ---
 .../nodes/physical/stream/StreamPhysicalRel.scala  |   55 -
 .../BatchCommonSubGraphBasedOptimizer.scala        |   87 -
 .../optimize/CommonSubGraphBasedOptimizer.scala    |  135 -
 .../flink/table/plan/optimize/Optimizer.scala      |   37 -
 .../flink/table/plan/optimize/RelNodeBlock.scala   |  444 ---
 .../StreamCommonSubGraphBasedOptimizer.scala       |  327 --
 .../optimize/program/BatchOptimizeContext.scala    |   26 -
 .../plan/optimize/program/FlinkBatchProgram.scala  |  211 --
 .../optimize/program/FlinkChainedProgram.scala     |  166 -
 .../optimize/program/FlinkDecorrelateProgram.scala |   97 -
 .../plan/optimize/program/FlinkGroupProgram.scala  |  107 -
 .../plan/optimize/program/FlinkHepProgram.scala    |  105 -
 .../optimize/program/FlinkHepRuleSetProgram.scala  |  196 --
 .../FlinkMiniBatchIntervalTraitInitProgram.scala   |   35 -
 .../optimize/program/FlinkOptimizeContext.scala    |   28 -
 .../optimize/program/FlinkOptimizeProgram.scala    |   36 -
 .../program/FlinkRelTimeIndicatorProgram.scala     |   36 -
 .../optimize/program/FlinkRuleSetProgram.scala     |   77 -
 .../plan/optimize/program/FlinkStreamProgram.scala |  208 --
 .../FlinkUpdateAsRetractionTraitInitProgram.scala  |   38 -
 .../optimize/program/FlinkVolcanoProgram.scala     |  132 -
 .../optimize/program/StreamOptimizeContext.scala   |   53 -
 .../plan/reuse/DeadlockBreakupProcessor.scala      |  345 ---
 .../flink/table/plan/reuse/SubplanReuser.scala     |  240 --
 .../table/plan/rules/FlinkBatchRuleSets.scala      |  404 ---
 .../table/plan/rules/FlinkStreamRuleSets.scala     |  415 ---
 .../logical/AggregateReduceGroupingRule.scala      |  124 -
 .../logical/BatchLogicalWindowAggregateRule.scala  |   80 -
 .../plan/rules/logical/CalcRankTransposeRule.scala |  197 --
 .../rules/logical/CalcSnapshotTransposeRule.scala  |   50 -
 .../rules/logical/ConvertToNotInOrInRule.scala     |  187 --
 .../rules/logical/DecomposeGroupingSetsRule.scala  |  374 ---
 .../logical/EnumerableToLogicalTableScan.scala     |   49 -
 .../plan/rules/logical/FlinkCalcMergeRule.scala    |  108 -
 .../plan/rules/logical/FlinkLimit0RemoveRule.scala |   50 -
 .../plan/rules/logical/FlinkLogicalRankRule.scala  |  270 --
 .../plan/rules/logical/FlinkPruneEmptyRules.scala  |   70 -
 .../rules/logical/FlinkRewriteSubQueryRule.scala   |  168 --
 .../rules/logical/FlinkSubQueryRemoveRule.scala    |  459 ---
 .../JoinConditionEqualityTransferRule.scala        |  172 --
 .../logical/JoinConditionTypeCoerceRule.scala      |  128 -
 .../JoinDependentConditionDerivationRule.scala     |  145 -
 .../rules/logical/JoinDeriveNullFilterRule.scala   |  107 -
 ...relateToJoinFromTemporalTableFunctionRule.scala |  230 --
 ...gicalCorrelateToJoinFromTemporalTableRule.scala |  129 -
 .../plan/rules/logical/LogicalUnnestRule.scala     |  181 --
 .../logical/LogicalWindowAggregateRuleBase.scala   |  212 --
 .../logical/ProjectSemiAntiJoinTransposeRule.scala |  167 -
 .../rules/logical/PruneAggregateCallRule.scala     |  201 --
 .../PushFilterIntoTableSourceScanRule.scala        |  151 -
 .../PushPartitionIntoTableSourceScanRule.scala     |  170 --
 .../PushProjectIntoTableSourceScanRule.scala       |  116 -
 .../rules/logical/RankNumberColumnRemoveRule.scala |   84 -
 .../logical/ReplaceIntersectWithSemiJoinRule.scala |   64 -
 .../logical/ReplaceMinusWithAntiJoinRule.scala     |   64 -
 .../plan/rules/logical/RewriteCoalesceRule.scala   |  237 --
 .../rules/logical/RewriteIntersectAllRule.scala    |  143 -
 .../plan/rules/logical/RewriteMinusAllRule.scala   |  121 -
 .../logical/RewriteMultiJoinConditionRule.scala    |  129 -
 .../logical/SimplifyFilterConditionRule.scala      |  103 -
 .../rules/logical/SimplifyJoinConditionRule.scala  |   70 -
 .../plan/rules/logical/SplitAggregateRule.scala    |  413 ---
 .../logical/StreamLogicalWindowAggregateRule.scala |   90 -
 .../WindowAggregateReduceFunctionsRule.scala       |   82 -
 .../rules/logical/WindowGroupReorderRule.scala     |  137 -
 .../plan/rules/logical/WindowPropertiesRule.scala  |  295 --
 .../rules/physical/FlinkExpandConversionRule.scala |  168 --
 .../physical/batch/BatchExecAggRuleBase.scala      |  173 --
 .../batch/BatchExecBoundedStreamScanRule.scala     |   58 -
 .../rules/physical/batch/BatchExecCalcRule.scala   |   55 -
 .../BatchExecConstantTableFunctionScanRule.scala   |   85 -
 .../physical/batch/BatchExecCorrelateRule.scala    |   85 -
 .../rules/physical/batch/BatchExecExpandRule.scala |   54 -
 .../physical/batch/BatchExecHashAggRule.scala      |  183 --
 .../physical/batch/BatchExecHashJoinRule.scala     |  195 --
 .../batch/BatchExecIntermediateTableScanRule.scala |   48 -
 .../physical/batch/BatchExecJoinRuleBase.scala     |  108 -
 .../rules/physical/batch/BatchExecLimitRule.scala  |  108 -
 .../physical/batch/BatchExecLookupJoinRule.scala   |  101 -
 .../batch/BatchExecNestedLoopJoinRule.scala        |   88 -
 .../batch/BatchExecNestedLoopJoinRuleBase.scala    |   65 -
 .../batch/BatchExecOverAggregateRule.scala         |  181 --
 .../rules/physical/batch/BatchExecRankRule.scala   |  119 -
 .../batch/BatchExecSingleRowJoinRule.scala         |   86 -
 .../rules/physical/batch/BatchExecSinkRule.scala   |   86 -
 .../physical/batch/BatchExecSortAggRule.scala      |  196 --
 .../physical/batch/BatchExecSortLimitRule.scala    |  109 -
 .../batch/BatchExecSortMergeJoinRule.scala         |  157 -
 .../rules/physical/batch/BatchExecSortRule.scala   |   90 -
 .../batch/BatchExecTableSourceScanRule.scala       |   68 -
 .../rules/physical/batch/BatchExecUnionRule.scala  |   60 -
 .../rules/physical/batch/BatchExecValuesRule.scala |   51 -
 .../batch/BatchExecWindowAggregateRule.scala       |  441 ---
 .../batch/RemoveRedundantLocalHashAggRule.scala    |   60 -
 .../batch/RemoveRedundantLocalRankRule.scala       |   60 -
 .../batch/RemoveRedundantLocalSortAggRule.scala    |  110 -
 .../physical/common/CommonLookupJoinRule.scala     |  152 -
 .../physical/stream/IncrementalAggregateRule.scala |  197 --
 .../stream/MiniBatchIntervalInferRule.scala        |  129 -
 .../rules/physical/stream/StreamExecCalcRule.scala |   55 -
 .../StreamExecConstantTableFunctionScanRule.scala  |   85 -
 .../physical/stream/StreamExecCorrelateRule.scala  |  129 -
 .../stream/StreamExecDataStreamScanRule.scala      |   61 -
 .../stream/StreamExecDeduplicateRule.scala         |  136 -
 .../physical/stream/StreamExecExpandRule.scala     |   55 -
 .../stream/StreamExecGroupAggregateRule.scala      |   81 -
 .../StreamExecGroupWindowAggregateRule.scala       |  102 -
 .../StreamExecIntermediateTableScanRule.scala      |   48 -
 .../rules/physical/stream/StreamExecJoinRule.scala |  140 -
 .../physical/stream/StreamExecLimitRule.scala      |   70 -
 .../physical/stream/StreamExecLookupJoinRule.scala |   98 -
 .../physical/stream/StreamExecMatchRule.scala      |  160 -
 .../stream/StreamExecOverAggregateRule.scala       |   79 -
 .../rules/physical/stream/StreamExecRankRule.scala |   75 -
 .../stream/StreamExecRetractionRules.scala         |  241 --
 .../rules/physical/stream/StreamExecSinkRule.scala |   85 -
 .../physical/stream/StreamExecSortLimitRule.scala  |   69 -
 .../rules/physical/stream/StreamExecSortRule.scala |   68 -
 .../stream/StreamExecTableSourceScanRule.scala     |   70 -
 .../stream/StreamExecTemporalJoinRule.scala        |  102 -
 .../stream/StreamExecTemporalSortRule.scala        |   91 -
 .../physical/stream/StreamExecUnionRule.scala      |   61 -
 .../physical/stream/StreamExecValuesRule.scala     |   53 -
 .../stream/StreamExecWatermarkAssignerRule.scala   |   64 -
 .../physical/stream/StreamExecWindowJoinRule.scala |  137 -
 .../stream/TwoStageOptimizedAggregateRule.scala    |  179 --
 .../flink/table/plan/schema/DataStreamTable.scala  |   77 -
 .../schema/DeferredTypeFlinkTableFunction.scala    |   64 -
 .../flink/table/plan/schema/FlinkRelOptTable.scala |  335 ---
 .../flink/table/plan/schema/FlinkTable.scala       |   52 -
 .../table/plan/schema/FlinkTableFunction.scala     |   61 -
 .../table/plan/schema/GenericRelDataType.scala     |   68 -
 .../flink/table/plan/schema/InlineTable.scala      |  124 -
 .../table/plan/schema/IntermediateRelTable.scala   |   64 -
 .../flink/table/plan/schema/TableSinkTable.scala   |   52 -
 .../table/plan/schema/TableSourceSinkTable.scala   |   75 -
 .../flink/table/plan/schema/TableSourceTable.scala |   80 -
 .../plan/schema/TimeIndicatorRelDataType.scala     |   54 -
 .../plan/schema/TypedFlinkTableFunction.scala      |   67 -
 .../flink/table/plan/stats/FlinkStatistic.scala    |  185 --
 .../flink/table/plan/stats/ValueInterval.scala     |  313 --
 .../table/plan/trait/FlinkRelDistribution.scala    |  258 --
 .../plan/trait/FlinkRelDistributionTraitDef.scala  |   48 -
 .../table/plan/trait/MiniBatchIntervalTrait.scala  |  100 -
 .../plan/trait/MiniBatchIntervalTraitDef.scala     |   48 -
 .../table/plan/trait/RelModifiedMonotonicity.scala |   42 -
 .../apache/flink/table/plan/trait/TraitUtil.scala  |   68 -
 .../table/plan/trait/retractionTraitDefs.scala     |   81 -
 .../flink/table/plan/trait/retractionTraits.scala  |  113 -
 .../flink/table/plan/util/AggFunctionFactory.scala |  636 ----
 .../flink/table/plan/util/AggregateUtil.scala      |  742 -----
 .../flink/table/plan/util/ColumnIntervalUtil.scala |  332 --
 .../flink/table/plan/util/CorrelateUtil.scala      |  137 -
 .../flink/table/plan/util/ExecNodePlanDumper.scala |  411 ---
 .../apache/flink/table/plan/util/ExpandUtil.scala  |  248 --
 .../table/plan/util/ExplodeFunctionUtil.scala      |  179 --
 .../flink/table/plan/util/FlinkRelMdUtil.scala     |  742 -----
 .../flink/table/plan/util/FlinkRelOptUtil.scala    |  606 ----
 .../flink/table/plan/util/FlinkRexUtil.scala       |  397 ---
 .../apache/flink/table/plan/util/JoinUtil.scala    |  140 -
 .../flink/table/plan/util/LookupJoinUtil.scala     |   47 -
 .../apache/flink/table/plan/util/MatchUtil.scala   |   54 -
 .../flink/table/plan/util/OverAggregateUtil.scala  |  113 -
 .../flink/table/plan/util/PartitionPruner.scala    |  169 --
 .../table/plan/util/RankProcessStrategy.scala      |  110 -
 .../apache/flink/table/plan/util/RankUtil.scala    |  308 --
 .../flink/table/plan/util/RelDigestUtil.scala      |  120 -
 .../flink/table/plan/util/RelExplainUtil.scala     |  872 ------
 .../apache/flink/table/plan/util/RelShuttles.scala |  216 --
 .../flink/table/plan/util/RelTreeWriterImpl.scala  |  129 -
 .../flink/table/plan/util/RexDefaultVisitor.scala  |   66 -
 .../flink/table/plan/util/RexNodeExtractor.scala   |  446 ---
 .../flink/table/plan/util/RexNodeRewriter.scala    |   62 -
 .../apache/flink/table/plan/util/ScanUtil.scala    |  132 -
 .../flink/table/plan/util/SetOpRewriteUtil.scala   |  118 -
 .../apache/flink/table/plan/util/SortUtil.scala    |  129 -
 .../flink/table/plan/util/TemporalJoinUtil.scala   |  105 -
 .../table/plan/util/UpdatingPlanChecker.scala      |  247 --
 .../flink/table/plan/util/WindowEmitStrategy.scala |  210 --
 .../flink/table/plan/util/WindowJoinUtil.scala     |  522 ----
 .../apache/flink/table/plan/util/aggregation.scala |  136 -
 .../apache/flink/table/planner/BatchPlanner.scala  |  120 -
 .../apache/flink/table/planner/PlannerBase.scala   |  293 --
 .../apache/flink/table/planner/StreamPlanner.scala |  121 -
 .../table/planner/calcite/CalciteConfig.scala      |  228 ++
 .../planner/calcite/FlinkCalciteSqlValidator.scala |   95 +
 .../flink/table/planner/calcite/FlinkContext.scala |   45 +
 .../table/planner/calcite/FlinkContextImpl.scala   |   32 +
 .../table/planner/calcite/FlinkLocalRef.scala      |   49 +
 .../planner/calcite/FlinkLogicalRelFactories.scala |  262 ++
 .../table/planner/calcite/FlinkPlannerImpl.scala   |  210 ++
 .../table/planner/calcite/FlinkRelBuilder.scala    |  169 ++
 .../table/planner/calcite/FlinkRelFactories.scala  |  124 +
 .../table/planner/calcite/FlinkRelOptCluster.scala |   77 +
 .../calcite/FlinkRelOptClusterFactory.scala        |   43 +
 .../table/planner/calcite/FlinkTypeFactory.scala   |  518 ++++
 .../table/planner/calcite/FlinkTypeSystem.scala    |  148 +
 .../planner/calcite/PreValidateReWriter.scala      |  201 ++
 .../calcite/RelTimeIndicatorConverter.scala        |  701 +++++
 .../table/planner/codegen/CalcCodeGenerator.scala  |  215 ++
 .../table/planner/codegen/CodeGenException.scala   |   24 +
 .../flink/table/planner/codegen/CodeGenUtils.scala |  721 +++++
 .../planner/codegen/CodeGeneratorContext.scala     |  830 +++++
 .../planner/codegen/CollectorCodeGenerator.scala   |   98 +
 .../planner/codegen/CorrelateCodeGenerator.scala   |  427 +++
 .../planner/codegen/EqualiserCodeGenerator.scala   |  147 +
 .../planner/codegen/ExpandCodeGenerator.scala      |   77 +
 .../table/planner/codegen/ExprCodeGenerator.scala  |  757 +++++
 .../table/planner/codegen/ExpressionReducer.scala  |  255 ++
 .../planner/codegen/FunctionCodeGenerator.scala    |  218 ++
 .../table/planner/codegen/GenerateUtils.scala      |  827 +++++
 .../planner/codegen/GeneratedExpression.scala      |  103 +
 .../table/planner/codegen/HashCodeGenerator.scala  |  106 +
 .../flink/table/planner/codegen/Indenter.scala     |   58 +
 .../planner/codegen/InputFormatCodeGenerator.scala |   93 +
 .../planner/codegen/LongHashJoinGenerator.scala    |  359 +++
 .../planner/codegen/LookupJoinCodeGenerator.scala  |  456 +++
 .../table/planner/codegen/MatchCodeGenerator.scala |  815 +++++
 .../codegen/NestedLoopJoinCodeGenerator.scala      |  370 +++
 .../planner/codegen/OperatorCodeGenerator.scala    |  232 ++
 .../planner/codegen/ProjectionCodeGenerator.scala  |  214 ++
 .../table/planner/codegen/SinkCodeGenerator.scala  |  276 ++
 .../planner/codegen/ValuesCodeGenerator.scala      |   60 +
 .../table/planner/codegen/agg/AggCodeGen.scala     |   49 +
 .../codegen/agg/AggsHandlerCodeGenerator.scala     |  825 +++++
 .../codegen/agg/DeclarativeAggCodeGen.scala        |  310 ++
 .../planner/codegen/agg/DistinctAggCodeGen.scala   |  935 ++++++
 .../planner/codegen/agg/ImperativeAggCodeGen.scala |  507 ++++
 .../codegen/agg/batch/AggCodeGenHelper.scala       |  717 +++++
 .../agg/batch/AggWithoutKeysCodeGenerator.scala    |  118 +
 .../codegen/agg/batch/HashAggCodeGenHelper.scala   |  847 ++++++
 .../codegen/agg/batch/HashAggCodeGenerator.scala   |  230 ++
 .../agg/batch/HashWindowCodeGenerator.scala        |  815 +++++
 .../codegen/agg/batch/SortAggCodeGenerator.scala   |  149 +
 .../agg/batch/SortWindowCodeGenerator.scala        |  269 ++
 .../codegen/agg/batch/WindowCodeGenerator.scala    |  765 +++++
 .../planner/codegen/calls/BuiltInMethods.scala     |  494 +++
 .../planner/codegen/calls/CallGenerator.scala      |   35 +
 .../planner/codegen/calls/ConstantCallGen.scala    |   37 +
 .../codegen/calls/CurrentTimePointCallGen.scala    |   58 +
 .../table/planner/codegen/calls/DivCallGen.scala   |   67 +
 .../planner/codegen/calls/ExtractCallGen.scala     |  115 +
 .../planner/codegen/calls/FloorCeilCallGen.scala   |   93 +
 .../planner/codegen/calls/FunctionGenerator.scala  |  923 ++++++
 .../planner/codegen/calls/HashCodeCallGen.scala    |   36 +
 .../table/planner/codegen/calls/IfCallGen.scala    |   67 +
 .../table/planner/codegen/calls/LikeCallGen.scala  |  172 ++
 .../planner/codegen/calls/MethodCallGen.scala      |   70 +
 .../table/planner/codegen/calls/PrintCallGen.scala |   67 +
 .../table/planner/codegen/calls/RandCallGen.scala  |   56 +
 .../codegen/calls/ScalarFunctionCallGen.scala      |  138 +
 .../planner/codegen/calls/ScalarOperatorGens.scala | 2177 ++++++++++++++
 .../planner/codegen/calls/StringCallGen.scala      |  813 +++++
 .../codegen/calls/TableFunctionCallGen.scala       |   93 +
 .../codegen/calls/TimestampDiffCallGen.scala       |  120 +
 ...ltiFieldRangeBoundComparatorCodeGenerator.scala |   84 +
 .../over/RangeBoundComparatorCodeGenerator.scala   |  177 ++
 .../codegen/sort/ComparatorCodeGenerator.scala     |   84 +
 .../planner/codegen/sort/SortCodeGenerator.scala   |  464 +++
 .../table/planner/dataview/DataViewSpec.scala      |   48 +
 .../table/planner/dataview/DataViewUtils.scala     |  215 ++
 .../table/planner/delegation/BatchPlanner.scala    |  119 +
 .../table/planner/delegation/PlannerBase.scala     |  294 ++
 .../table/planner/delegation/StreamPlanner.scala   |  120 +
 .../planner/expressions/ExpressionBridge.scala     |   41 +
 .../table/planner/expressions/InputTypeSpec.scala  |   69 +
 .../planner/expressions/PlannerExpression.scala    |   88 +
 .../expressions/PlannerExpressionConverter.scala   |  849 ++++++
 .../expressions/PlannerExpressionUtils.scala       |   68 +
 .../table/planner/expressions/Reinterpret.scala    |   45 +
 .../planner/expressions/ReturnTypeInference.scala  |  220 ++
 .../table/planner/expressions/aggregations.scala   |  230 ++
 .../table/planner/expressions/arithmetic.scala     |  149 +
 .../flink/table/planner/expressions/call.scala     |  223 ++
 .../flink/table/planner/expressions/cast.scala     |   44 +
 .../table/planner/expressions/collection.scala     |  174 ++
 .../table/planner/expressions/comparison.scala     |  181 ++
 .../table/planner/expressions/composite.scala      |  100 +
 .../planner/expressions/fieldExpression.scala      |  230 ++
 .../planner/expressions/hashExpressions.scala      |   93 +
 .../flink/table/planner/expressions/literals.scala |   82 +
 .../flink/table/planner/expressions/logic.scala    |   87 +
 .../planner/expressions/mathExpressions.scala      |  401 +++
 .../flink/table/planner/expressions/ordering.scala |   43 +
 .../table/planner/expressions/overOffsets.scala    |   54 +
 .../table/{ => planner}/expressions/package.scala  |    0
 .../expressions/plannerWindowProperties.scala      |   86 +
 .../planner/expressions/stringExpressions.scala    |  490 +++
 .../flink/table/planner/expressions/subquery.scala |   76 +
 .../flink/table/planner/expressions/symbols.scala  |  127 +
 .../flink/table/planner/expressions/time.scala     |  322 ++
 .../planner/expressions/windowProperties.scala     |   61 +
 .../planner/functions/utils/AggSqlFunction.scala   |  216 ++
 .../functions/utils/ScalarSqlFunction.scala        |  199 ++
 .../planner/functions/utils/TableSqlFunction.scala |  191 ++
 .../functions/utils/UserDefinedFunctionUtils.scala |  858 ++++++
 .../org/apache/flink/table/planner/package.scala   |   43 +
 .../apache/flink/table/planner/plan/TreeNode.scala |  115 +
 .../flink/table/planner/plan/cost/FlinkCost.scala  |  354 +++
 .../table/planner/plan/cost/FlinkCostBase.scala    |   37 +
 .../table/planner/plan/cost/FlinkCostFactory.scala |   49 +
 .../planner/plan/cost/FlinkCostFactoryBase.scala   |   35 +
 .../planner/plan/logical/MatchRecognize.scala      |   42 +
 .../table/planner/plan/logical/groupWindows.scala  |   79 +
 .../metadata/AggCallSelectivityEstimator.scala     |  455 +++
 .../metadata/FlinkDefaultRelMetadataProvider.scala |   53 +
 .../plan/metadata/FlinkRelMdColumnInterval.scala   |  746 +++++
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |  295 ++
 .../metadata/FlinkRelMdColumnOriginNullCount.scala |  134 +
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |  739 +++++
 .../plan/metadata/FlinkRelMdCumulativeCost.scala   |   59 +
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |  626 ++++
 .../plan/metadata/FlinkRelMdDistribution.scala     |  103 +
 .../FlinkRelMdFilteredColumnInterval.scala         |  264 ++
 .../metadata/FlinkRelMdModifiedMonotonicity.scala  |  552 ++++
 .../metadata/FlinkRelMdNonCumulativeCost.scala     |   55 +
 .../FlinkRelMdPercentageOriginalRows.scala         |  146 +
 .../plan/metadata/FlinkRelMdPopulationSize.scala   |  412 +++
 .../planner/plan/metadata/FlinkRelMdRowCount.scala |  456 +++
 .../plan/metadata/FlinkRelMdSelectivity.scala      |  301 ++
 .../planner/plan/metadata/FlinkRelMdSize.scala     |  436 +++
 .../plan/metadata/FlinkRelMdUniqueGroups.scala     |  432 +++
 .../plan/metadata/FlinkRelMdUniqueKeys.scala       |  566 ++++
 .../plan/metadata/SelectivityEstimator.scala       | 1146 +++++++
 .../planner/plan/nodes/FlinkConventions.scala      |   55 +
 .../table/planner/plan/nodes/FlinkRelNode.scala    |  118 +
 .../table/planner/plan/nodes/calcite/Expand.scala  |  109 +
 .../planner/plan/nodes/calcite/LogicalExpand.scala |   58 +
 .../planner/plan/nodes/calcite/LogicalRank.scala   |   96 +
 .../planner/plan/nodes/calcite/LogicalSink.scala   |   57 +
 .../nodes/calcite/LogicalWatermarkAssigner.scala   |   43 +
 .../nodes/calcite/LogicalWindowAggregate.scala     |   89 +
 .../table/planner/plan/nodes/calcite/Rank.scala    |  137 +
 .../table/planner/plan/nodes/calcite/Sink.scala    |   58 +
 .../plan/nodes/calcite/WatermarkAssigner.scala     |   65 +
 .../plan/nodes/calcite/WindowAggregate.scala       |   87 +
 .../planner/plan/nodes/common/CommonCalc.scala     |   87 +
 .../nodes/common/CommonIntermediateTableScan.scala |   54 +
 .../plan/nodes/common/CommonLookupJoin.scala       |  678 +++++
 .../plan/nodes/common/CommonPhysicalExchange.scala |  109 +
 .../plan/nodes/common/CommonPhysicalJoin.scala     |   86 +
 .../planner/plan/nodes/exec/BatchExecNode.scala    |   35 +
 .../table/planner/plan/nodes/exec/ExecNode.scala   |   97 +
 .../planner/plan/nodes/exec/StreamExecNode.scala   |   27 +
 .../plan/nodes/logical/FlinkLogicalAggregate.scala |  162 +
 .../plan/nodes/logical/FlinkLogicalCalc.scala      |   79 +
 .../plan/nodes/logical/FlinkLogicalCorrelate.scala |   99 +
 .../logical/FlinkLogicalDataStreamTableScan.scala  |  100 +
 .../plan/nodes/logical/FlinkLogicalExpand.scala    |   82 +
 .../FlinkLogicalIntermediateTableScan.scala        |   92 +
 .../plan/nodes/logical/FlinkLogicalIntersect.scala |   89 +
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |  108 +
 .../plan/nodes/logical/FlinkLogicalMatch.scala     |  133 +
 .../plan/nodes/logical/FlinkLogicalMinus.scala     |   87 +
 .../nodes/logical/FlinkLogicalOverAggregate.scala  |  106 +
 .../plan/nodes/logical/FlinkLogicalRank.scala      |  123 +
 .../plan/nodes/logical/FlinkLogicalRel.scala       |   28 +
 .../plan/nodes/logical/FlinkLogicalSink.scala      |   80 +
 .../plan/nodes/logical/FlinkLogicalSnapshot.scala  |   90 +
 .../plan/nodes/logical/FlinkLogicalSort.scala      |  139 +
 .../logical/FlinkLogicalTableFunctionScan.scala    |  125 +
 .../logical/FlinkLogicalTableSourceScan.scala      |  124 +
 .../plan/nodes/logical/FlinkLogicalUnion.scala     |   93 +
 .../plan/nodes/logical/FlinkLogicalValues.scala    |   89 +
 .../logical/FlinkLogicalWatermarkAssigner.scala    |   79 +
 .../logical/FlinkLogicalWindowAggregate.scala      |  119 +
 .../plan/nodes/physical/FlinkPhysicalRel.scala     |   42 +
 .../nodes/physical/PhysicalTableSourceScan.scala   |   55 +
 .../batch/BatchExecBoundedStreamScan.scala         |  118 +
 .../plan/nodes/physical/batch/BatchExecCalc.scala  |  166 +
 .../nodes/physical/batch/BatchExecCorrelate.scala  |  207 ++
 .../nodes/physical/batch/BatchExecExchange.scala   |  196 ++
 .../nodes/physical/batch/BatchExecExpand.scala     |  111 +
 .../batch/BatchExecGroupAggregateBase.scala        |   94 +
 .../physical/batch/BatchExecHashAggregate.scala    |  155 +
 .../batch/BatchExecHashAggregateBase.scala         |  157 +
 .../nodes/physical/batch/BatchExecHashJoin.scala   |  285 ++
 .../batch/BatchExecHashWindowAggregate.scala       |   98 +
 .../batch/BatchExecHashWindowAggregateBase.scala   |  159 +
 .../batch/BatchExecIntermediateTableScan.scala     |   45 +
 .../nodes/physical/batch/BatchExecJoinBase.scala   |  225 ++
 .../plan/nodes/physical/batch/BatchExecLimit.scala |  121 +
 .../batch/BatchExecLocalHashAggregate.scala        |  132 +
 .../batch/BatchExecLocalHashWindowAggregate.scala  |   93 +
 .../batch/BatchExecLocalSortAggregate.scala        |  141 +
 .../batch/BatchExecLocalSortWindowAggregate.scala  |   91 +
 .../nodes/physical/batch/BatchExecLookupJoin.scala |  100 +
 .../physical/batch/BatchExecNestedLoopJoin.scala   |  184 ++
 .../physical/batch/BatchExecOverAggregate.scala    |  631 ++++
 .../plan/nodes/physical/batch/BatchExecRank.scala  |  304 ++
 .../plan/nodes/physical/batch/BatchExecSink.scala  |  191 ++
 .../plan/nodes/physical/batch/BatchExecSort.scala  |  138 +
 .../physical/batch/BatchExecSortAggregate.scala    |  170 ++
 .../batch/BatchExecSortAggregateBase.scala         |  130 +
 .../nodes/physical/batch/BatchExecSortLimit.scala  |  158 +
 .../physical/batch/BatchExecSortMergeJoin.scala    |  287 ++
 .../batch/BatchExecSortWindowAggregate.scala       |   96 +
 .../batch/BatchExecSortWindowAggregateBase.scala   |  145 +
 .../physical/batch/BatchExecTableSourceScan.scala  |  159 +
 .../plan/nodes/physical/batch/BatchExecUnion.scala |  119 +
 .../nodes/physical/batch/BatchExecValues.scala     |   88 +
 .../batch/BatchExecWindowAggregateBase.scala       |   83 +
 .../nodes/physical/batch/BatchPhysicalRel.scala    |   29 +
 .../nodes/physical/stream/StreamExecCalc.scala     |  132 +
 .../physical/stream/StreamExecCorrelate.scala      |  143 +
 .../physical/stream/StreamExecDataStreamScan.scala |  176 ++
 .../physical/stream/StreamExecDeduplicate.scala    |  166 +
 .../nodes/physical/stream/StreamExecExchange.scala |  114 +
 .../nodes/physical/stream/StreamExecExpand.scala   |  110 +
 .../stream/StreamExecGlobalGroupAggregate.scala    |  227 ++
 .../physical/stream/StreamExecGroupAggregate.scala |  220 ++
 .../stream/StreamExecGroupAggregateBase.scala      |   54 +
 .../stream/StreamExecGroupWindowAggregate.scala    |  365 +++
 .../StreamExecIncrementalGroupAggregate.scala      |  221 ++
 .../stream/StreamExecIntermediateTableScan.scala   |   62 +
 .../nodes/physical/stream/StreamExecJoin.scala     |  258 ++
 .../nodes/physical/stream/StreamExecLimit.scala    |  183 ++
 .../stream/StreamExecLocalGroupAggregate.scala     |  160 +
 .../physical/stream/StreamExecLookupJoin.scala     |  110 +
 .../nodes/physical/stream/StreamExecMatch.scala    |  463 +++
 .../physical/stream/StreamExecOverAggregate.scala  |  469 +++
 .../nodes/physical/stream/StreamExecRank.scala     |  254 ++
 .../nodes/physical/stream/StreamExecSink.scala     |  240 ++
 .../nodes/physical/stream/StreamExecSort.scala     |  154 +
 .../physical/stream/StreamExecSortLimit.scala      |  238 ++
 .../stream/StreamExecTableSourceScan.scala         |  254 ++
 .../physical/stream/StreamExecTemporalJoin.scala   |  426 +++
 .../physical/stream/StreamExecTemporalSort.scala   |  204 ++
 .../nodes/physical/stream/StreamExecUnion.scala    |   90 +
 .../nodes/physical/stream/StreamExecValues.scala   |   92 +
 .../stream/StreamExecWatermarkAssigner.scala       |  185 ++
 .../physical/stream/StreamExecWindowJoin.scala     |  360 +++
 .../nodes/physical/stream/StreamPhysicalRel.scala  |   55 +
 .../BatchCommonSubGraphBasedOptimizer.scala        |   87 +
 .../optimize/CommonSubGraphBasedOptimizer.scala    |  135 +
 .../table/planner/plan/optimize/Optimizer.scala    |   37 +
 .../table/planner/plan/optimize/RelNodeBlock.scala |  444 +++
 .../StreamCommonSubGraphBasedOptimizer.scala       |  329 ++
 .../optimize/program/BatchOptimizeContext.scala    |   26 +
 .../plan/optimize/program/FlinkBatchProgram.scala  |  211 ++
 .../optimize/program/FlinkChainedProgram.scala     |  166 +
 .../optimize/program/FlinkDecorrelateProgram.scala |   97 +
 .../plan/optimize/program/FlinkGroupProgram.scala  |  107 +
 .../plan/optimize/program/FlinkHepProgram.scala    |  105 +
 .../optimize/program/FlinkHepRuleSetProgram.scala  |  196 ++
 .../FlinkMiniBatchIntervalTraitInitProgram.scala   |   35 +
 .../optimize/program/FlinkOptimizeContext.scala    |   28 +
 .../optimize/program/FlinkOptimizeProgram.scala    |   36 +
 .../program/FlinkRelTimeIndicatorProgram.scala     |   36 +
 .../optimize/program/FlinkRuleSetProgram.scala     |   77 +
 .../plan/optimize/program/FlinkStreamProgram.scala |  208 ++
 .../FlinkUpdateAsRetractionTraitInitProgram.scala  |   38 +
 .../optimize/program/FlinkVolcanoProgram.scala     |  132 +
 .../optimize/program/StreamOptimizeContext.scala   |   53 +
 .../plan/reuse/DeadlockBreakupProcessor.scala      |  345 +++
 .../table/planner/plan/reuse/SubplanReuser.scala   |  241 ++
 .../planner/plan/rules/FlinkBatchRuleSets.scala    |  404 +++
 .../planner/plan/rules/FlinkStreamRuleSets.scala   |  415 +++
 .../logical/AggregateReduceGroupingRule.scala      |  124 +
 .../logical/BatchLogicalWindowAggregateRule.scala  |   80 +
 .../plan/rules/logical/CalcRankTransposeRule.scala |  197 ++
 .../rules/logical/CalcSnapshotTransposeRule.scala  |   51 +
 .../rules/logical/ConvertToNotInOrInRule.scala     |  187 ++
 .../rules/logical/DecomposeGroupingSetsRule.scala  |  374 +++
 .../logical/EnumerableToLogicalTableScan.scala     |   49 +
 .../plan/rules/logical/FlinkCalcMergeRule.scala    |  108 +
 .../plan/rules/logical/FlinkLimit0RemoveRule.scala |   50 +
 .../plan/rules/logical/FlinkLogicalRankRule.scala  |  270 ++
 .../plan/rules/logical/FlinkPruneEmptyRules.scala  |   70 +
 .../rules/logical/FlinkRewriteSubQueryRule.scala   |  168 ++
 .../rules/logical/FlinkSubQueryRemoveRule.scala    |  459 +++
 .../JoinConditionEqualityTransferRule.scala        |  172 ++
 .../logical/JoinConditionTypeCoerceRule.scala      |  128 +
 .../JoinDependentConditionDerivationRule.scala     |  145 +
 .../rules/logical/JoinDeriveNullFilterRule.scala   |  107 +
 ...relateToJoinFromTemporalTableFunctionRule.scala |  231 ++
 ...gicalCorrelateToJoinFromTemporalTableRule.scala |  130 +
 .../plan/rules/logical/LogicalUnnestRule.scala     |  181 ++
 .../logical/LogicalWindowAggregateRuleBase.scala   |  213 ++
 .../logical/ProjectSemiAntiJoinTransposeRule.scala |  167 +
 .../rules/logical/PruneAggregateCallRule.scala     |  201 ++
 .../PushFilterIntoTableSourceScanRule.scala        |  152 +
 .../PushPartitionIntoTableSourceScanRule.scala     |  170 ++
 .../PushProjectIntoTableSourceScanRule.scala       |  116 +
 .../rules/logical/RankNumberColumnRemoveRule.scala |   84 +
 .../logical/ReplaceIntersectWithSemiJoinRule.scala |   64 +
 .../logical/ReplaceMinusWithAntiJoinRule.scala     |   64 +
 .../plan/rules/logical/RewriteCoalesceRule.scala   |  237 ++
 .../rules/logical/RewriteIntersectAllRule.scala    |  143 +
 .../plan/rules/logical/RewriteMinusAllRule.scala   |  121 +
 .../logical/RewriteMultiJoinConditionRule.scala    |  129 +
 .../logical/SimplifyFilterConditionRule.scala      |  103 +
 .../rules/logical/SimplifyJoinConditionRule.scala  |   70 +
 .../plan/rules/logical/SplitAggregateRule.scala    |  414 +++
 .../logical/StreamLogicalWindowAggregateRule.scala |   90 +
 .../WindowAggregateReduceFunctionsRule.scala       |   82 +
 .../rules/logical/WindowGroupReorderRule.scala     |  137 +
 .../plan/rules/logical/WindowPropertiesRule.scala  |  296 ++
 .../rules/physical/FlinkExpandConversionRule.scala |  168 ++
 .../physical/batch/BatchExecAggRuleBase.scala      |  173 ++
 .../batch/BatchExecBoundedStreamScanRule.scala     |   58 +
 .../rules/physical/batch/BatchExecCalcRule.scala   |   55 +
 .../BatchExecConstantTableFunctionScanRule.scala   |   85 +
 .../physical/batch/BatchExecCorrelateRule.scala    |   86 +
 .../rules/physical/batch/BatchExecExpandRule.scala |   55 +
 .../physical/batch/BatchExecHashAggRule.scala      |  183 ++
 .../physical/batch/BatchExecHashJoinRule.scala     |  196 ++
 .../batch/BatchExecIntermediateTableScanRule.scala |   48 +
 .../physical/batch/BatchExecJoinRuleBase.scala     |  107 +
 .../rules/physical/batch/BatchExecLimitRule.scala  |  108 +
 .../physical/batch/BatchExecLookupJoinRule.scala   |  101 +
 .../batch/BatchExecNestedLoopJoinRule.scala        |   88 +
 .../batch/BatchExecNestedLoopJoinRuleBase.scala    |   65 +
 .../batch/BatchExecOverAggregateRule.scala         |  181 ++
 .../rules/physical/batch/BatchExecRankRule.scala   |  119 +
 .../batch/BatchExecSingleRowJoinRule.scala         |   86 +
 .../rules/physical/batch/BatchExecSinkRule.scala   |   88 +
 .../physical/batch/BatchExecSortAggRule.scala      |  196 ++
 .../physical/batch/BatchExecSortLimitRule.scala    |  109 +
 .../batch/BatchExecSortMergeJoinRule.scala         |  157 +
 .../rules/physical/batch/BatchExecSortRule.scala   |   90 +
 .../batch/BatchExecTableSourceScanRule.scala       |   69 +
 .../rules/physical/batch/BatchExecUnionRule.scala  |   61 +
 .../rules/physical/batch/BatchExecValuesRule.scala |   52 +
 .../batch/BatchExecWindowAggregateRule.scala       |  442 +++
 .../batch/RemoveRedundantLocalHashAggRule.scala    |   60 +
 .../batch/RemoveRedundantLocalRankRule.scala       |   60 +
 .../batch/RemoveRedundantLocalSortAggRule.scala    |  110 +
 .../physical/common/CommonLookupJoinRule.scala     |  153 +
 .../physical/stream/IncrementalAggregateRule.scala |  197 ++
 .../stream/MiniBatchIntervalInferRule.scala        |  129 +
 .../rules/physical/stream/StreamExecCalcRule.scala |   55 +
 .../StreamExecConstantTableFunctionScanRule.scala  |   85 +
 .../physical/stream/StreamExecCorrelateRule.scala  |  129 +
 .../stream/StreamExecDataStreamScanRule.scala      |   61 +
 .../stream/StreamExecDeduplicateRule.scala         |  136 +
 .../physical/stream/StreamExecExpandRule.scala     |   55 +
 .../stream/StreamExecGroupAggregateRule.scala      |   81 +
 .../StreamExecGroupWindowAggregateRule.scala       |  102 +
 .../StreamExecIntermediateTableScanRule.scala      |   48 +
 .../rules/physical/stream/StreamExecJoinRule.scala |  141 +
 .../physical/stream/StreamExecLimitRule.scala      |   70 +
 .../physical/stream/StreamExecLookupJoinRule.scala |   98 +
 .../physical/stream/StreamExecMatchRule.scala      |  161 +
 .../stream/StreamExecOverAggregateRule.scala       |   79 +
 .../rules/physical/stream/StreamExecRankRule.scala |   75 +
 .../stream/StreamExecRetractionRules.scala         |  241 ++
 .../rules/physical/stream/StreamExecSinkRule.scala |   86 +
 .../physical/stream/StreamExecSortLimitRule.scala  |   69 +
 .../rules/physical/stream/StreamExecSortRule.scala |   67 +
 .../stream/StreamExecTableSourceScanRule.scala     |   70 +
 .../stream/StreamExecTemporalJoinRule.scala        |  102 +
 .../stream/StreamExecTemporalSortRule.scala        |   91 +
 .../physical/stream/StreamExecUnionRule.scala      |   61 +
 .../physical/stream/StreamExecValuesRule.scala     |   53 +
 .../stream/StreamExecWatermarkAssignerRule.scala   |   64 +
 .../physical/stream/StreamExecWindowJoinRule.scala |  136 +
 .../stream/TwoStageOptimizedAggregateRule.scala    |  179 ++
 .../planner/plan/schema/DataStreamTable.scala      |   77 +
 .../schema/DeferredTypeFlinkTableFunction.scala    |   64 +
 .../planner/plan/schema/FlinkRelOptTable.scala     |  335 +++
 .../table/planner/plan/schema/FlinkTable.scala     |   53 +
 .../planner/plan/schema/FlinkTableFunction.scala   |   61 +
 .../planner/plan/schema/GenericRelDataType.scala   |   69 +
 .../table/planner/plan/schema/InlineTable.scala    |  124 +
 .../planner/plan/schema/IntermediateRelTable.scala |   64 +
 .../table/planner/plan/schema/TableSinkTable.scala |   52 +
 .../planner/plan/schema/TableSourceSinkTable.scala |   75 +
 .../planner/plan/schema/TableSourceTable.scala     |   81 +
 .../plan/schema/TimeIndicatorRelDataType.scala     |   54 +
 .../plan/schema/TypedFlinkTableFunction.scala      |   67 +
 .../table/planner/plan/stats/FlinkStatistic.scala  |  186 ++
 .../table/planner/plan/stats/ValueInterval.scala   |  313 ++
 .../planner/plan/trait/FlinkRelDistribution.scala  |  258 ++
 .../plan/trait/FlinkRelDistributionTraitDef.scala  |   48 +
 .../plan/trait/MiniBatchIntervalTrait.scala        |  100 +
 .../plan/trait/MiniBatchIntervalTraitDef.scala     |   48 +
 .../plan/trait/RelModifiedMonotonicity.scala       |   42 +
 .../flink/table/planner/plan/trait/TraitUtil.scala |   68 +
 .../planner/plan/trait/retractionTraitDefs.scala   |   81 +
 .../planner/plan/trait/retractionTraits.scala      |  113 +
 .../planner/plan/utils/AggFunctionFactory.scala    |  636 ++++
 .../table/planner/plan/utils/AggregateUtil.scala   |  747 +++++
 .../planner/plan/utils/ColumnIntervalUtil.scala    |  332 ++
 .../table/planner/plan/utils/CorrelateUtil.scala   |  137 +
 .../planner/plan/utils/ExecNodePlanDumper.scala    |  411 +++
 .../table/planner/plan/utils/ExpandUtil.scala      |  248 ++
 .../planner/plan/utils/ExplodeFunctionUtil.scala   |  178 ++
 .../table/planner/plan/utils/FlinkRelMdUtil.scala  |  742 +++++
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |  606 ++++
 .../table/planner/plan/utils/FlinkRexUtil.scala    |  397 +++
 .../flink/table/planner/plan/utils/JoinUtil.scala  |  141 +
 .../table/planner/plan/utils/LookupJoinUtil.scala  |   48 +
 .../flink/table/planner/plan/utils/MatchUtil.scala |   55 +
 .../planner/plan/utils/OverAggregateUtil.scala     |  114 +
 .../table/planner/plan/utils/PartitionPruner.scala |  169 ++
 .../planner/plan/utils/RankProcessStrategy.scala   |  110 +
 .../flink/table/planner/plan/utils/RankUtil.scala  |  308 ++
 .../table/planner/plan/utils/RelDigestUtil.scala   |  120 +
 .../table/planner/plan/utils/RelExplainUtil.scala  |  873 ++++++
 .../table/planner/plan/utils/RelShuttles.scala     |  216 ++
 .../planner/plan/utils/RelTreeWriterImpl.scala     |  129 +
 .../planner/plan/utils/RexDefaultVisitor.scala     |   66 +
 .../planner/plan/utils/RexNodeExtractor.scala      |  445 +++
 .../table/planner/plan/utils/RexNodeRewriter.scala |   62 +
 .../flink/table/planner/plan/utils/ScanUtil.scala  |  134 +
 .../planner/plan/utils/SetOpRewriteUtil.scala      |  118 +
 .../flink/table/planner/plan/utils/SortUtil.scala  |  129 +
 .../planner/plan/utils/TemporalJoinUtil.scala      |  106 +
 .../planner/plan/utils/UpdatingPlanChecker.scala   |  247 ++
 .../planner/plan/utils/WindowEmitStrategy.scala    |  210 ++
 .../table/planner/plan/utils/WindowJoinUtil.scala  |  522 ++++
 .../table/planner/plan/utils/aggregation.scala     |  136 +
 .../table/planner/sinks/CollectTableSink.scala     |   84 +
 .../table/planner/sinks/DataStreamTableSink.scala  |   69 +
 .../flink/table/planner/sinks/TableSinkUtils.scala |  107 +
 .../table/planner/sources/TableSourceUtil.scala    |  388 +++
 .../table/planner/typeutils/TypeCoercion.scala     |  166 +
 .../typeutils/TypeInfoCheckUtils.scala             |    0
 .../planner/utils/JavaScalaConversionUtil.scala    |   71 +
 .../apache/flink/table/planner/utils/Logging.scala |   28 +
 .../flink/table/planner/utils/PlanUtil.scala       |   81 +
 .../{ => planner}/validate/ValidationResult.scala  |    0
 .../flink/table/sinks/CollectTableSink.scala       |   83 -
 .../flink/table/sinks/DataStreamTableSink.scala    |   68 -
 .../apache/flink/table/sinks/TableSinkUtils.scala  |  106 -
 .../flink/table/sources/TableSourceUtil.scala      |  387 ---
 .../flink/table/typeutils/TypeCoercion.scala       |  166 -
 .../flink/table/util/JavaScalaConversionUtil.scala |   71 -
 .../org/apache/flink/table/util/Logging.scala      |   28 -
 .../org/apache/flink/table/util/PlanUtil.scala     |   81 -
 .../calcite/FlinkCalciteCatalogReaderTest.java     |   84 -
 .../table/codegen/LongHashJoinGeneratorTest.java   |   92 -
 .../flink/table/codegen/SortCodeGeneratorTest.java |  583 ----
 .../flink/table/codegen/agg/TestLongAvgFunc.java   |   74 -
 .../aggfunctions/AggFunctionTestBase.java          |  284 --
 .../ConcatWithRetractAggFunctionTest.java          |   80 -
 .../ConcatWsWithRetractAggFunctionTest.java        |  159 -
 ...FirstLastValueAggFunctionWithOrderTestBase.java |  147 -
 .../FirstValueAggFunctionWithOrderTest.java        |  466 ---
 .../FirstValueAggFunctionWithoutOrderTest.java     |  351 ---
 ...stValueWithRetractAggFunctionWithOrderTest.java |  474 ---
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 ---
 .../LastValueAggFunctionWithOrderTest.java         |  467 ---
 .../LastValueAggFunctionWithoutOrderTest.java      |  352 ---
 ...stValueWithRetractAggFunctionWithOrderTest.java |  480 ---
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 ---
 .../MaxWithRetractAggFunctionTest.java             |  564 ----
 .../MinWithRetractAggFunctionTest.java             |  556 ----
 .../plan/nodes/resource/MockNodeTestBase.java      |  185 --
 .../parallelism/FinalParallelismSetterTest.java    |  118 -
 .../parallelism/ShuffleStageGeneratorTest.java     |  323 --
 .../ShuffleStageParallelismCalculatorTest.java     |  123 -
 .../plan/util/JavaUserDefinedAggFunctions.java     |  421 ---
 .../planner/codegen/LongHashJoinGeneratorTest.java |   92 +
 .../planner/codegen/SortCodeGeneratorTest.java     |  583 ++++
 .../table/planner/codegen/agg/TestLongAvgFunc.java |   74 +
 .../aggfunctions/AggFunctionTestBase.java          |  284 ++
 .../ConcatWithRetractAggFunctionTest.java          |   80 +
 .../ConcatWsWithRetractAggFunctionTest.java        |  159 +
 ...FirstLastValueAggFunctionWithOrderTestBase.java |  147 +
 .../FirstValueAggFunctionWithOrderTest.java        |  466 +++
 .../FirstValueAggFunctionWithoutOrderTest.java     |  351 +++
 ...stValueWithRetractAggFunctionWithOrderTest.java |  474 +++
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 +++
 .../LastValueAggFunctionWithOrderTest.java         |  467 +++
 .../LastValueAggFunctionWithoutOrderTest.java      |  352 +++
 ...stValueWithRetractAggFunctionWithOrderTest.java |  480 +++
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 +++
 .../MaxWithRetractAggFunctionTest.java             |  564 ++++
 .../MinWithRetractAggFunctionTest.java             |  556 ++++
 .../plan/FlinkCalciteCatalogReaderTest.java        |   86 +
 .../plan/nodes/resource/MockNodeTestBase.java      |  185 ++
 .../parallelism/FinalParallelismSetterTest.java    |  118 +
 .../parallelism/ShuffleStageGeneratorTest.java     |  323 ++
 .../ShuffleStageParallelismCalculatorTest.java     |  123 +
 .../plan/utils/JavaUserDefinedAggFunctions.java    |  421 +++
 .../runtime/utils/BatchAbstractTestBase.java       |   54 +
 .../runtime/utils/FailingCollectionSource.java     |  251 ++
 .../table/planner/runtime/utils/JavaPojos.java     |   45 +
 .../runtime/utils/JavaUserDefinedAggFunctions.java |  428 +++
 .../utils/JavaUserDefinedScalarFunctions.java      |  157 +
 .../utils/JavaUserDefinedTableFunctions.java       |  142 +
 .../planner/runtime/utils/RangeInputFormat.java    |   67 +
 .../flink/table/planner/utils/BaseRowTestUtil.java |  108 +
 .../flink/table/planner/utils/DiffRepository.java  |  784 +++++
 .../table/runtime/utils/BatchAbstractTestBase.java |   54 -
 .../runtime/utils/FailingCollectionSource.java     |  251 --
 .../flink/table/runtime/utils/JavaPojos.java       |   45 -
 .../runtime/utils/JavaUserDefinedAggFunctions.java |  428 ---
 .../utils/JavaUserDefinedScalarFunctions.java      |  157 -
 .../utils/JavaUserDefinedTableFunctions.java       |  142 -
 .../table/runtime/utils/RangeInputFormat.java      |   67 -
 .../apache/flink/table/util/BaseRowTestUtil.java   |  108 -
 .../apache/flink/table/util/DiffRepository.java    |  784 -----
 .../org.apache.flink.table.factories.TableFactory  |    2 +-
 .../table/plan/batch/sql/SetOperatorsTest.xml      |  282 --
 .../flink/table/plan/batch/table/CalcTest.xml      |  235 --
 .../flink/table/plan/batch/table/CorrelateTest.xml |  130 -
 .../flink/table/plan/batch/table/JoinTest.xml      |  280 --
 .../table/plan/batch/table/SetOperatorsTest.xml    |  201 --
 .../stringexpr/CorrelateStringExpressionTest.xml   |  156 -
 .../PushFilterIntoTableSourceScanRuleTest.xml      |  172 --
 .../rules/logical/RewriteIntersectAllRuleTest.xml  |  151 -
 .../plan/rules/logical/RewriteMinusAllRuleTest.xml |  151 -
 .../table/plan/stream/sql/SetOperatorsTest.xml     |  282 --
 .../flink/table/plan/stream/table/CalcTest.xml     |  187 --
 .../plan/stream/table/ColumnFunctionsTest.xml      |  229 --
 .../table/plan/stream/table/CorrelateTest.xml      |  209 --
 .../table/plan/stream/table/OverWindowTest.xml     |  308 --
 .../stream => planner/plan/batch}/sql/CalcTest.xml |    0
 .../plan/batch/sql/DagOptimizationTest.xml         |    0
 .../plan/batch/sql/DeadlockBreakupTest.xml         |    0
 .../{ => planner}/plan/batch/sql/LimitTest.xml     |    0
 .../{ => planner}/plan/batch/sql/RankTest.xml      |    0
 .../plan/batch/sql/RemoveCollationTest.xml         |    0
 .../plan/batch/sql/RemoveShuffleTest.xml           |    0
 .../planner/plan/batch/sql/SetOperatorsTest.xml    |  282 ++
 .../{ => planner}/plan/batch/sql/SinkTest.xml      |    0
 .../{ => planner}/plan/batch/sql/SortLimitTest.xml |    0
 .../{ => planner}/plan/batch/sql/SortTest.xml      |    0
 .../plan/batch/sql/SubplanReuseTest.xml            |    0
 .../{ => planner}/plan/batch/sql/TableScanTest.xml |    0
 .../plan/batch/sql/TableSourceTest.xml             |    0
 .../plan/batch}/sql/UnionTest.xml                  |    0
 .../{ => planner}/plan/batch/sql/UnnestTest.xml    |    0
 .../{ => planner}/plan/batch/sql/ValuesTest.xml    |    0
 .../batch/sql/agg/AggregateReduceGroupingTest.xml  |    0
 .../plan/batch/sql/agg/DistinctAggregateTest.xml   |    0
 .../plan/batch/sql/agg/GroupingSetsTest.xml        |    0
 .../plan/batch/sql/agg/HashAggregateTest.xml       |    0
 .../plan/batch/sql/agg/OverAggregateTest.xml       |    0
 .../plan/batch/sql/agg/SortAggregateTest.xml       |    0
 .../plan/batch/sql/agg/WindowAggregateTest.xml     |    0
 .../plan/batch/sql/join/BroadcastHashJoinTest.xml  |    0
 .../sql/join/BroadcastHashSemiAntiJoinTest.xml     |    0
 .../plan/batch/sql/join/JoinReorderTest.xml        |    0
 .../plan/batch/sql/join/LookupJoinTest.xml         |    0
 .../plan/batch/sql/join/NestedLoopJoinTest.xml     |    0
 .../batch/sql/join/NestedLoopSemiAntiJoinTest.xml  |    0
 .../plan/batch/sql/join/SemiAntiJoinTest.xml       |    0
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   |    0
 .../sql/join/ShuffledHashSemiAntiJoinTest.xml      |    0
 .../plan/batch/sql/join/SingleRowJoinTest.xml      |    0
 .../plan/batch/sql/join/SortMergeJoinTest.xml      |    0
 .../batch/sql/join/SortMergeSemiAntiJoinTest.xml   |    0
 .../plan/batch/table/AggregateTest.xml             |    0
 .../table/planner/plan/batch/table/CalcTest.xml    |  235 ++
 .../plan/batch/table/ColumnFunctionsTest.xml       |    0
 .../planner/plan/batch/table/CorrelateTest.xml     |  130 +
 .../plan/batch/table/GroupWindowTest.xml           |    0
 .../table/planner/plan/batch/table/JoinTest.xml    |  280 ++
 .../planner/plan/batch/table/SetOperatorsTest.xml  |  201 ++
 .../stringexpr/CorrelateStringExpressionTest.xml   |  156 +
 .../batch/table/stringexpr/SetOperatorsTest.xml    |    0
 .../plan/nodes/resource/ExecNodeResourceTest.xml   |    0
 .../logical/AggregateReduceGroupingRuleTest.xml    |    0
 .../logical/CalcPruneAggregateCallRuleTest.xml     |    0
 .../rules/logical/CalcRankTransposeRuleTest.xml    |    0
 .../rules/logical/ConvertToNotInOrInRuleTest.xml   |    0
 .../logical/DecomposeGroupingSetsRuleTest.xml      |    0
 .../rules/logical/ExpressionReductionRulesTest.xml |    0
 ...nkAggregateExpandDistinctAggregatesRuleTest.xml |    0
 .../FlinkAggregateJoinTransposeRuleTest.xml        |    0
 .../rules/logical/FlinkAggregateRemoveRuleTest.xml |    0
 .../plan/rules/logical/FlinkCalcMergeRuleTest.xml  |    0
 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml |    0
 .../logical/FlinkJoinPushExpressionsRuleTest.xml   |    0
 .../rules/logical/FlinkLimit0RemoveRuleTest.xml    |    0
 .../FlinkLogicalRankRuleForConstantRangeTest.xml   |    0
 .../FlinkLogicalRankRuleForRangeEndTest.xml        |    0
 .../rules/logical/FlinkPruneEmptyRulesTest.xml     |    0
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.xml   |    0
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.xml     |    0
 .../FlinkSemiAntiJoinProjectTransposeRuleTest.xml  |    0
 .../JoinConditionEqualityTransferRuleTest.xml      |    0
 .../logical/JoinConditionTypeCoerceRuleTest.xml    |    0
 .../JoinDependentConditionDerivationRuleTest.xml   |    0
 .../rules/logical/JoinDeriveNullFilterRuleTest.xml |    0
 .../plan/rules/logical/LogicalUnnestRuleTest.xml   |    0
 .../logical/ProjectPruneAggregateCallRuleTest.xml  |    0
 .../ProjectSemiAntiJoinTransposeRuleTest.xml       |    0
 .../PushFilterIntoTableSourceScanRuleTest.xml      |  172 ++
 .../PushPartitionIntoTableSourceScanRuleTest.xml   |    0
 .../PushProjectIntoTableSourceScanRuleTest.xml     |    0
 .../logical/RankNumberColumnRemoveRuleTest.xml     |    0
 .../ReplaceIntersectWithSemiJoinRuleTest.xml       |    0
 .../logical/ReplaceMinusWithAntiJoinRuleTest.xml   |    0
 .../plan/rules/logical/RewriteCoalesceRuleTest.xml |    0
 .../rules/logical/RewriteIntersectAllRuleTest.xml  |  151 +
 .../plan/rules/logical/RewriteMinusAllRuleTest.xml |  151 +
 .../logical/RewriteMultiJoinConditionRuleTest.xml  |    0
 .../logical/SimplifyFilterConditionRuleTest.xml    |    0
 .../logical/SimplifyJoinConditionRuleTest.xml      |    0
 .../plan/rules/logical/SplitAggregateRuleTest.xml  |    0
 .../rules/logical/WindowGroupReorderRuleTest.xml   |    0
 .../subquery/FlinkRewriteSubQueryRuleTest.xml      |    0
 .../logical/subquery/SubQueryAntiJoinTest.xml      |    0
 .../logical/subquery/SubQuerySemiJoinTest.xml      |    0
 .../SubqueryCorrelateVariablesValidationTest.xml   |    0
 .../batch/RemoveRedundantLocalHashAggRuleTest.xml  |    0
 .../batch/RemoveRedundantLocalRankRuleTest.xml     |    0
 .../batch/RemoveRedundantLocalSortAggRuleTest.xml  |    0
 .../rules/physical/stream/RetractionRulesTest.xml  |    0
 .../stream/RetractionRulesWithTwoStageAggTest.xml  |    0
 .../batch => planner/plan/stream}/sql/CalcTest.xml |    0
 .../plan/stream/sql/DagOptimizationTest.xml        |    0
 .../{ => planner}/plan/stream/sql/LimitTest.xml    |    0
 .../plan/stream/sql/MiniBatchIntervalInferTest.xml |    0
 .../plan/stream/sql/ModifiedMonotonicityTest.xml   |    0
 .../{ => planner}/plan/stream/sql/RankTest.xml     |    0
 .../stream/sql/RelTimeIndicatorConverterTest.xml   |    0
 .../planner/plan/stream/sql/SetOperatorsTest.xml   |  282 ++
 .../{ => planner}/plan/stream/sql/SinkTest.xml     |    0
 .../plan/stream/sql/SortLimitTest.xml              |    0
 .../{ => planner}/plan/stream/sql/SortTest.xml     |    0
 .../plan/stream/sql/SubplanReuseTest.xml           |    0
 .../plan/stream/sql/TableScanTest.xml              |    0
 .../plan/stream/sql/TableSourceTest.xml            |    0
 .../plan/stream}/sql/UnionTest.xml                 |    0
 .../{ => planner}/plan/stream/sql/UnnestTest.xml   |    0
 .../{ => planner}/plan/stream/sql/ValuesTest.xml   |    0
 .../plan/stream/sql/agg/AggregateTest.xml          |    0
 .../plan/stream/sql/agg/DistinctAggregateTest.xml  |    0
 .../plan/stream/sql/agg/GroupingSetsTest.xml       |    0
 .../stream/sql/agg/IncrementalAggregateTest.xml    |    0
 .../plan/stream/sql/agg/OverAggregateTest.xml      |    0
 .../plan/stream/sql/agg/TwoStageAggregateTest.xml  |    0
 .../plan/stream/sql/agg/WindowAggregateTest.xml    |    0
 .../plan/stream/sql/join/JoinReorderTest.xml       |    0
 .../plan/stream/sql/join/JoinTest.xml              |    0
 .../plan/stream/sql/join/LookupJoinTest.xml        |    0
 .../plan/stream/sql/join/SemiAntiJoinTest.xml      |    0
 .../plan/stream/sql/join/TemporalJoinTest.xml      |    0
 .../plan/stream/sql/join/WindowJoinTest.xml        |    0
 .../plan/stream/table/AggregateTest.xml            |    0
 .../table/planner/plan/stream/table/CalcTest.xml   |  187 ++
 .../plan/stream/table/ColumnFunctionsTest.xml      |  229 ++
 .../planner/plan/stream/table/CorrelateTest.xml    |  209 ++
 .../plan/stream/table/GroupWindowTest.xml          |    0
 .../{ => planner}/plan/stream/table/JoinTest.xml   |    0
 .../planner/plan/stream/table/OverWindowTest.xml   |  308 ++
 .../plan/stream/table/SetOperatorsTest.xml         |    0
 .../plan/stream/table/TableSourceTest.xml          |    0
 .../plan/stream/table/TwoStageAggregateTest.xml    |    0
 .../flink/table/api/TableEnvironmentTest.scala     |    2 +-
 .../apache/flink/table/api/batch/ExplainTest.scala |    4 +-
 .../flink/table/api/stream/ExplainTest.scala       |    4 +-
 .../validation/MatchRecognizeValidationTest.scala  |    6 +-
 .../sql/validation/OverWindowValidationTest.scala  |    5 +-
 .../table/calcite/CalciteConfigBuilderTest.scala   |  248 --
 .../flink/table/calcite/FlinkTypeFactoryTest.scala |   85 -
 .../flink/table/catalog/CatalogTableITCase.scala   |  534 ----
 .../table/codegen/HashCodeGeneratorTest.scala      |   62 -
 .../codegen/ProjectionCodeGeneratorTest.scala      |  110 -
 .../flink/table/codegen/agg/AggTestBase.scala      |  112 -
 .../codegen/agg/AggsHandlerCodeGeneratorTest.scala |  102 -
 .../codegen/agg/batch/AggWithoutKeysTest.scala     |  104 -
 .../table/codegen/agg/batch/BatchAggTestBase.scala |   99 -
 .../agg/batch/HashAggCodeGeneratorTest.scala       |  131 -
 .../agg/batch/SortAggCodeGeneratorTest.scala       |  120 -
 .../flink/table/expressions/ArrayTypeTest.scala    |  215 --
 .../table/expressions/CompositeAccessTest.scala    |  141 -
 .../flink/table/expressions/DecimalTypeTest.scala  |  222 --
 .../flink/table/expressions/KeywordParseTest.scala |   62 -
 .../flink/table/expressions/LiteralTest.scala      |  157 -
 .../flink/table/expressions/MapTypeTest.scala      |  193 --
 .../table/expressions/MathFunctionsTest.scala      |  693 -----
 .../table/expressions/NonDeterministicTests.scala  |   79 -
 .../flink/table/expressions/RowTypeTest.scala      |   88 -
 .../table/expressions/ScalarFunctionsTest.scala    | 3181 --------------------
 .../table/expressions/ScalarOperatorsTest.scala    |  125 -
 .../table/expressions/SqlExpressionTest.scala      |  314 --
 .../table/expressions/TemporalTypesTest.scala      |  756 -----
 .../expressions/utils/ArrayTypeTestBase.scala      |   63 -
 .../expressions/utils/CompositeTypeTestBase.scala  |   92 -
 .../expressions/utils/ExpressionTestBase.scala     |  211 --
 .../table/expressions/utils/MapTypeTestBase.scala  |   75 -
 .../table/expressions/utils/RowTypeTestBase.scala  |   67 -
 .../utils/ScalarOperatorsTestBase.scala            |   75 -
 .../expressions/utils/ScalarTypesTestBase.scala    |  148 -
 .../utils/userDefinedScalarFunctions.scala         |  412 ---
 .../validation/ArrayTypeValidationTest.scala       |   58 -
 .../validation/CompositeAccessValidationTest.scala |   38 -
 .../validation/MapTypeValidationTest.scala         |   46 -
 .../validation/RowTypeValidationTest.scala         |   41 -
 .../validation/ScalarFunctionsValidationTest.scala |  149 -
 .../utils/TestCollectionTableFactory.scala         |  269 --
 .../flink/table/match/PatternTranslatorTest.scala  |  397 ---
 .../table/match/PatternTranslatorTestBase.scala    |  140 -
 .../flink/table/plan/batch/sql/CalcTest.scala      |  161 -
 .../table/plan/batch/sql/DagOptimizationTest.scala |  471 ---
 .../table/plan/batch/sql/DeadlockBreakupTest.scala |  184 --
 .../flink/table/plan/batch/sql/LimitTest.scala     |   93 -
 .../flink/table/plan/batch/sql/RankTest.scala      |  171 --
 .../table/plan/batch/sql/RemoveCollationTest.scala |  386 ---
 .../table/plan/batch/sql/RemoveShuffleTest.scala   |  548 ----
 .../table/plan/batch/sql/SetOperatorsTest.scala    |  130 -
 .../flink/table/plan/batch/sql/SinkTest.scala      |   91 -
 .../flink/table/plan/batch/sql/SortLimitTest.scala |   95 -
 .../flink/table/plan/batch/sql/SortTest.scala      |   75 -
 .../table/plan/batch/sql/SubplanReuseTest.scala    |  446 ---
 .../flink/table/plan/batch/sql/TableScanTest.scala |   37 -
 .../table/plan/batch/sql/TableSourceTest.scala     |  224 --
 .../flink/table/plan/batch/sql/UnionTest.scala     |   66 -
 .../flink/table/plan/batch/sql/UnnestTest.scala    |   26 -
 .../flink/table/plan/batch/sql/ValuesTest.scala    |   49 -
 .../sql/agg/AggregateReduceGroupingTest.scala      |   24 -
 .../plan/batch/sql/agg/AggregateTestBase.scala     |  203 --
 .../plan/batch/sql/agg/DistinctAggregateTest.scala |   85 -
 .../plan/batch/sql/agg/GroupingSetsTest.scala      |  458 ---
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   73 -
 .../plan/batch/sql/agg/OverAggregateTest.scala     |  344 ---
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   55 -
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |  314 --
 .../batch/sql/join/BroadcastHashJoinTest.scala     |  140 -
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |  179 --
 .../plan/batch/sql/join/JoinReorderTest.scala      |   25 -
 .../table/plan/batch/sql/join/JoinTestBase.scala   |  212 --
 .../table/plan/batch/sql/join/LookupJoinTest.scala |  281 --
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   32 -
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   32 -
 .../plan/batch/sql/join/SemiAntiJoinTest.scala     |   26 -
 .../plan/batch/sql/join/SemiAntiJoinTestBase.scala |  585 ----
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |  118 -
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |  185 --
 .../plan/batch/sql/join/SingleRowJoinTest.scala    |   96 -
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |  110 -
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |  162 -
 .../plan/batch/sql/join/TemporalJoinTest.scala     |  112 -
 .../table/plan/batch/table/AggregateTest.scala     |   75 -
 .../flink/table/plan/batch/table/CalcTest.scala    |  203 --
 .../plan/batch/table/ColumnFunctionsTest.scala     |   52 -
 .../table/plan/batch/table/CorrelateTest.scala     |  120 -
 .../table/plan/batch/table/GroupWindowTest.scala   |  158 -
 .../flink/table/plan/batch/table/JoinTest.scala    |  212 --
 .../table/plan/batch/table/SetOperatorsTest.scala  |  133 -
 .../plan/batch/table/TemporalTableJoinTest.scala   |   72 -
 .../stringexpr/AggregateStringExpressionTest.scala |  341 ---
 .../stringexpr/CalcStringExpressionTest.scala      |  366 ---
 .../stringexpr/CorrelateStringExpressionTest.scala |   94 -
 .../stringexpr/JoinStringExpressionTest.scala      |  187 --
 .../batch/table/stringexpr/SetOperatorsTest.scala  |   52 -
 .../stringexpr/SortStringExpressionTest.scala      |   61 -
 .../table/validation/AggregateValidationTest.scala |  221 --
 .../table/validation/CalcValidationTest.scala      |  117 -
 .../table/validation/CorrelateValidationTest.scala |   46 -
 .../validation/GroupWindowValidationTest.scala     |  172 --
 .../table/validation/JoinValidationTest.scala      |  118 -
 .../validation/OverWindowValidationTest.scala      |   56 -
 .../validation/SetOperatorsValidationTest.scala    |  113 -
 .../table/validation/SortValidationTest.scala      |   69 -
 .../common/AggregateReduceGroupingTestBase.scala   |  314 --
 .../table/plan/common/JoinReorderTestBase.scala    |  233 --
 .../flink/table/plan/common/UnnestTestBase.scala   |  129 -
 .../flink/table/plan/cost/FlinkCostTest.scala      |  187 --
 .../metadata/AggCallSelectivityEstimatorTest.scala |  630 ----
 .../metadata/FlinkRelMdColumnIntervalTest.scala    |  608 ----
 .../metadata/FlinkRelMdColumnNullCountTest.scala   |  287 --
 .../FlinkRelMdColumnOriginNullCountTest.scala      |  144 -
 .../metadata/FlinkRelMdColumnUniquenessTest.scala  |  611 ----
 .../metadata/FlinkRelMdCumulativeCostTest.scala    |   52 -
 .../metadata/FlinkRelMdDistinctRowCountTest.scala  |  640 ----
 .../plan/metadata/FlinkRelMdDistributionTest.scala |  109 -
 .../FlinkRelMdFilteredColumnIntervalTest.scala     |  187 --
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  | 2305 --------------
 .../FlinkRelMdModifiedMonotonicityTest.scala       |  290 --
 .../metadata/FlinkRelMdNonCumulativeCostTest.scala |   51 -
 .../FlinkRelMdPercentageOriginalRowsTest.scala     |   88 -
 .../metadata/FlinkRelMdPopulationSizeTest.scala    |  355 ---
 .../plan/metadata/FlinkRelMdRowCountTest.scala     |  253 --
 .../plan/metadata/FlinkRelMdSelectivityTest.scala  |  535 ----
 .../table/plan/metadata/FlinkRelMdSizeTest.scala   |  209 --
 .../plan/metadata/FlinkRelMdUniqueGroupsTest.scala |  646 ----
 .../plan/metadata/FlinkRelMdUniqueKeysTest.scala   |  273 --
 .../metadata/MetadataHandlerConsistencyTest.scala  |  150 -
 .../table/plan/metadata/MetadataTestUtil.scala     |  269 --
 .../plan/metadata/SelectivityEstimatorTest.scala   | 1068 -------
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  249 --
 .../optimize/program/FlinkChainedProgramTest.scala |  157 -
 .../program/FlinkHepRuleSetProgramTest.scala       |  103 -
 .../optimize/program/FlinkVolcanoProgramTest.scala |   51 -
 .../logical/AggregateReduceGroupingRuleTest.scala  |   44 -
 .../logical/CalcPruneAggregateCallRuleTest.scala   |   50 -
 .../rules/logical/CalcRankTransposeRuleTest.scala  |  191 --
 .../rules/logical/ConvertToNotInOrInRuleTest.scala |  168 --
 .../logical/DecomposeGroupingSetsRuleTest.scala    |  145 -
 .../logical/ExpressionReductionRulesTest.scala     |   49 -
 ...AggregateExpandDistinctAggregatesRuleTest.scala |  174 --
 .../FlinkAggregateJoinTransposeRuleTest.scala      |  147 -
 .../logical/FlinkAggregateRemoveRuleTest.scala     |  235 --
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   85 -
 .../rules/logical/FlinkFilterJoinRuleTest.scala    |  158 -
 .../logical/FlinkJoinPushExpressionsRuleTest.scala |   80 -
 .../rules/logical/FlinkLimit0RemoveRuleTest.scala  |   99 -
 .../FlinkLogicalRankRuleForConstantRangeTest.scala |  211 --
 .../FlinkLogicalRankRuleForRangeEndTest.scala      |  196 --
 .../rules/logical/FlinkPruneEmptyRulesTest.scala   |   71 -
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.scala |   76 -
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.scala   |  359 ---
 ...FlinkSemiAntiJoinProjectTransposeRuleTest.scala |   81 -
 .../JoinConditionEqualityTransferRuleTest.scala    |  146 -
 .../logical/JoinConditionTypeCoerceRuleTest.scala  |  116 -
 .../JoinDependentConditionDerivationRuleTest.scala |  120 -
 .../logical/JoinDeriveNullFilterRuleTest.scala     |  118 -
 .../plan/rules/logical/LogicalUnnestRuleTest.scala |   50 -
 .../ProjectPruneAggregateCallRuleTest.scala        |   50 -
 .../ProjectSemiAntiJoinTransposeRuleTest.scala     |  151 -
 .../logical/PruneAggregateCallRuleTestBase.scala   |  176 --
 .../PushFilterIntoTableSourceScanRuleTest.scala    |   97 -
 .../PushPartitionIntoTableSourceScanRuleTest.scala |   96 -
 .../PushProjectIntoTableSourceScanRuleTest.scala   |  127 -
 .../logical/RankNumberColumnRemoveRuleTest.scala   |   97 -
 .../ReplaceIntersectWithSemiJoinRuleTest.scala     |   74 -
 .../logical/ReplaceMinusWithAntiJoinRuleTest.scala |   74 -
 .../rules/logical/RewriteCoalesceRuleTest.scala    |  141 -
 .../logical/RewriteIntersectAllRuleTest.scala      |   75 -
 .../rules/logical/RewriteMinusAllRuleTest.scala    |   73 -
 .../RewriteMultiJoinConditionRuleTest.scala        |  151 -
 .../logical/SimplifyFilterConditionRuleTest.scala  |  108 -
 .../logical/SimplifyJoinConditionRuleTest.scala    |   68 -
 .../rules/logical/SplitAggregateRuleTest.scala     |  186 --
 .../rules/logical/WindowGroupReorderRuleTest.scala |  180 --
 .../subquery/FlinkRewriteSubQueryRuleTest.scala    |  212 --
 .../logical/subquery/SubQueryAntiJoinTest.scala    |  770 -----
 .../logical/subquery/SubQuerySemiJoinTest.scala    | 1674 ----------
 .../rules/logical/subquery/SubQueryTestBase.scala  |   44 -
 .../SubqueryCorrelateVariablesValidationTest.scala |  130 -
 .../RemoveRedundantLocalHashAggRuleTest.scala      |   72 -
 .../batch/RemoveRedundantLocalRankRuleTest.scala   |   73 -
 .../RemoveRedundantLocalSortAggRuleTest.scala      |   67 -
 .../physical/stream/RetractionRulesTest.scala      |   70 -
 .../RetractionRulesWithTwoStageAggTest.scala       |   76 -
 .../plan/schema/TimeIndicatorRelDataTypeTest.scala |   42 -
 .../flink/table/plan/stats/ValueIntervalTest.scala |  459 ---
 .../flink/table/plan/stream/sql/CalcTest.scala     |  159 -
 .../plan/stream/sql/DagOptimizationTest.scala      |  530 ----
 .../flink/table/plan/stream/sql/LimitTest.scala    |  213 --
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  351 ---
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  276 --
 .../flink/table/plan/stream/sql/RankTest.scala     |  718 -----
 .../stream/sql/RelTimeIndicatorConverterTest.scala |  178 --
 .../table/plan/stream/sql/SetOperatorsTest.scala   |  128 -
 .../flink/table/plan/stream/sql/SinkTest.scala     |  153 -
 .../table/plan/stream/sql/SortLimitTest.scala      |  327 --
 .../flink/table/plan/stream/sql/SortTest.scala     |   79 -
 .../table/plan/stream/sql/SubplanReuseTest.scala   |  300 --
 .../table/plan/stream/sql/TableScanTest.scala      |   42 -
 .../table/plan/stream/sql/TableSourceTest.scala    |  395 ---
 .../flink/table/plan/stream/sql/UnionTest.scala    |   66 -
 .../flink/table/plan/stream/sql/UnnestTest.scala   |   26 -
 .../flink/table/plan/stream/sql/ValuesTest.scala   |   49 -
 .../table/plan/stream/sql/agg/AggregateTest.scala  |  262 --
 .../stream/sql/agg/DistinctAggregateTest.scala     |  224 --
 .../plan/stream/sql/agg/GroupingSetsTest.scala     |  458 ---
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   52 -
 .../plan/stream/sql/agg/OverAggregateTest.scala    |  433 ---
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   79 -
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  298 --
 .../plan/stream/sql/join/JoinReorderTest.scala     |   25 -
 .../table/plan/stream/sql/join/JoinTest.scala      |  284 --
 .../plan/stream/sql/join/LookupJoinTest.scala      |  493 ---
 .../plan/stream/sql/join/SemiAntiJoinTest.scala    |  576 ----
 .../plan/stream/sql/join/TemporalJoinTest.scala    |  132 -
 .../plan/stream/sql/join/WindowJoinTest.scala      |  453 ---
 .../table/plan/stream/table/AggregateTest.scala    |  230 --
 .../flink/table/plan/stream/table/CalcTest.scala   |  161 -
 .../plan/stream/table/ColumnFunctionsTest.scala    |  229 --
 .../table/plan/stream/table/CorrelateTest.scala    |  181 --
 .../table/plan/stream/table/GroupWindowTest.scala  |  409 ---
 .../flink/table/plan/stream/table/JoinTest.scala   |  263 --
 .../table/plan/stream/table/OverWindowTest.scala   |  222 --
 .../table/plan/stream/table/SetOperatorsTest.scala |   87 -
 .../table/plan/stream/table/TableSourceTest.scala  |  302 --
 .../plan/stream/table/TemporalTableJoinTest.scala  |  191 --
 .../plan/stream/table/TwoStageAggregateTest.scala  |  111 -
 .../stringexpr/AggregateStringExpressionTest.scala |  246 --
 .../stringexpr/CalcStringExpressionTest.scala      |  183 --
 .../stringexpr/CorrelateStringExpressionTest.scala |  160 -
 .../GroupWindowStringExpressionTest.scala          |  263 --
 ...pWindowTableAggregateStringExpressionTest.scala |  227 --
 .../OverWindowStringExpressionTest.scala           |  248 --
 .../SetOperatorsStringExpressionTest.scala         |   50 -
 .../table/validation/AggregateValidationTest.scala |  127 -
 .../table/validation/CalcValidationTest.scala      |  165 -
 .../table/validation/CorrelateValidationTest.scala |  178 --
 .../validation/GroupWindowValidationTest.scala     |  308 --
 .../validation/OverWindowValidationTest.scala      |  164 -
 .../validation/SetOperatorsValidationTest.scala    |   80 -
 .../table/validation/TableSinkValidationTest.scala |   87 -
 .../TemporalTableJoinValidationTest.scala          |  114 -
 .../validation/UnsupportedOpsValidationTest.scala  |  109 -
 .../plan/trait/FlinkRelDistributionTest.scala      |  165 -
 .../table/plan/util/FlinkRelOptUtilTest.scala      |  141 -
 .../flink/table/plan/util/FlinkRexUtilTest.scala   |  407 ---
 .../flink/table/plan/util/InputTypeBuilder.scala   |   53 -
 .../table/plan/util/PartitionPrunerTest.scala      |  111 -
 .../flink/table/plan/util/RelDigestUtilTest.scala  |  108 -
 .../table/plan/util/RexNodeExtractorTest.scala     |  903 ------
 .../table/plan/util/RexNodeRewriterTest.scala      |   75 -
 .../flink/table/plan/util/RexNodeTestBase.scala    |   90 -
 .../org/apache/flink/table/plan/util/pojos.scala   |   51 -
 .../planner/calcite/CalciteConfigBuilderTest.scala |  249 ++
 .../planner/calcite/FlinkTypeFactoryTest.scala     |   86 +
 .../table/planner/catalog/CatalogTableITCase.scala |  535 ++++
 .../planner/codegen/HashCodeGeneratorTest.scala    |   62 +
 .../codegen/ProjectionCodeGeneratorTest.scala      |  110 +
 .../table/planner/codegen/agg/AggTestBase.scala    |  112 +
 .../codegen/agg/AggsHandlerCodeGeneratorTest.scala |  102 +
 .../codegen/agg/batch/AggWithoutKeysTest.scala     |  104 +
 .../codegen/agg/batch/BatchAggTestBase.scala       |   99 +
 .../agg/batch/HashAggCodeGeneratorTest.scala       |  131 +
 .../agg/batch/SortAggCodeGeneratorTest.scala       |  120 +
 .../table/planner/expressions/ArrayTypeTest.scala  |  216 ++
 .../planner/expressions/CompositeAccessTest.scala  |  142 +
 .../planner/expressions/DecimalTypeTest.scala      |  223 ++
 .../planner/expressions/KeywordParseTest.scala     |   63 +
 .../table/planner/expressions/LiteralTest.scala    |  158 +
 .../table/planner/expressions/MapTypeTest.scala    |  194 ++
 .../planner/expressions/MathFunctionsTest.scala    |  694 +++++
 .../expressions/NonDeterministicTests.scala        |   80 +
 .../table/planner/expressions/RowTypeTest.scala    |   89 +
 .../planner/expressions/ScalarFunctionsTest.scala  | 3181 ++++++++++++++++++++
 .../planner/expressions/ScalarOperatorsTest.scala  |  126 +
 .../planner/expressions/SqlExpressionTest.scala    |  314 ++
 .../planner/expressions/TemporalTypesTest.scala    |  756 +++++
 .../expressions/utils/ArrayTypeTestBase.scala      |   63 +
 .../expressions/utils/CompositeTypeTestBase.scala  |   92 +
 .../expressions/utils/ExpressionTestBase.scala     |  211 ++
 .../expressions/utils/MapTypeTestBase.scala        |   76 +
 .../expressions/utils/RowTypeTestBase.scala        |   67 +
 .../utils/ScalarOperatorsTestBase.scala            |   75 +
 .../expressions/utils/ScalarTypesTestBase.scala    |  149 +
 .../utils/userDefinedScalarFunctions.scala         |  412 +++
 .../validation/ArrayTypeValidationTest.scala       |   59 +
 .../validation/CompositeAccessValidationTest.scala |   39 +
 .../validation/MapTypeValidationTest.scala         |   47 +
 .../validation/RowTypeValidationTest.scala         |   42 +
 .../validation/ScalarFunctionsValidationTest.scala |  150 +
 .../utils/TestCollectionTableFactory.scala         |  269 ++
 .../planner/match/PatternTranslatorTest.scala      |  398 +++
 .../planner/match/PatternTranslatorTestBase.scala  |  140 +
 .../table/planner/plan/batch/sql/CalcTest.scala    |  161 +
 .../plan/batch/sql/DagOptimizationTest.scala       |  471 +++
 .../plan/batch/sql/DeadlockBreakupTest.scala       |  184 ++
 .../table/planner/plan/batch/sql/LimitTest.scala   |   93 +
 .../table/planner/plan/batch/sql/RankTest.scala    |  171 ++
 .../plan/batch/sql/RemoveCollationTest.scala       |  388 +++
 .../planner/plan/batch/sql/RemoveShuffleTest.scala |  550 ++++
 .../planner/plan/batch/sql/SetOperatorsTest.scala  |  131 +
 .../table/planner/plan/batch/sql/SinkTest.scala    |   93 +
 .../planner/plan/batch/sql/SortLimitTest.scala     |   95 +
 .../table/planner/plan/batch/sql/SortTest.scala    |   75 +
 .../planner/plan/batch/sql/SubplanReuseTest.scala  |  446 +++
 .../planner/plan/batch/sql/TableScanTest.scala     |   37 +
 .../planner/plan/batch/sql/TableSourceTest.scala   |  223 ++
 .../table/planner/plan/batch/sql/UnionTest.scala   |   66 +
 .../table/planner/plan/batch/sql/UnnestTest.scala  |   26 +
 .../table/planner/plan/batch/sql/ValuesTest.scala  |   49 +
 .../sql/agg/AggregateReduceGroupingTest.scala      |   24 +
 .../plan/batch/sql/agg/AggregateTestBase.scala     |  203 ++
 .../plan/batch/sql/agg/DistinctAggregateTest.scala |   85 +
 .../plan/batch/sql/agg/GroupingSetsTest.scala      |  458 +++
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   74 +
 .../plan/batch/sql/agg/OverAggregateTest.scala     |  344 +++
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   55 +
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |  315 ++
 .../batch/sql/join/BroadcastHashJoinTest.scala     |  141 +
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |  180 ++
 .../plan/batch/sql/join/JoinReorderTest.scala      |   25 +
 .../planner/plan/batch/sql/join/JoinTestBase.scala |  212 ++
 .../plan/batch/sql/join/LookupJoinTest.scala       |  282 ++
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   32 +
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   32 +
 .../plan/batch/sql/join/SemiAntiJoinTest.scala     |   26 +
 .../plan/batch/sql/join/SemiAntiJoinTestBase.scala |  585 ++++
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |  119 +
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |  186 ++
 .../plan/batch/sql/join/SingleRowJoinTest.scala    |   96 +
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |  111 +
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |  163 +
 .../plan/batch/sql/join/TemporalJoinTest.scala     |  112 +
 .../planner/plan/batch/table/AggregateTest.scala   |   75 +
 .../table/planner/plan/batch/table/CalcTest.scala  |  203 ++
 .../plan/batch/table/ColumnFunctionsTest.scala     |   52 +
 .../planner/plan/batch/table/CorrelateTest.scala   |  120 +
 .../planner/plan/batch/table/GroupWindowTest.scala |  158 +
 .../table/planner/plan/batch/table/JoinTest.scala  |  212 ++
 .../plan/batch/table/SetOperatorsTest.scala        |  133 +
 .../plan/batch/table/TemporalTableJoinTest.scala   |   72 +
 .../stringexpr/AggregateStringExpressionTest.scala |  341 +++
 .../stringexpr/CalcStringExpressionTest.scala      |  366 +++
 .../stringexpr/CorrelateStringExpressionTest.scala |   94 +
 .../stringexpr/JoinStringExpressionTest.scala      |  187 ++
 .../batch/table/stringexpr/SetOperatorsTest.scala  |   52 +
 .../stringexpr/SortStringExpressionTest.scala      |   61 +
 .../table/validation/AggregateValidationTest.scala |  221 ++
 .../table/validation/CalcValidationTest.scala      |  117 +
 .../table/validation/CorrelateValidationTest.scala |   46 +
 .../validation/GroupWindowValidationTest.scala     |  172 ++
 .../table/validation/JoinValidationTest.scala      |  118 +
 .../validation/OverWindowValidationTest.scala      |   56 +
 .../validation/SetOperatorsValidationTest.scala    |  113 +
 .../table/validation/SortValidationTest.scala      |   69 +
 .../common/AggregateReduceGroupingTestBase.scala   |  315 ++
 .../planner/plan/common/JoinReorderTestBase.scala  |  235 ++
 .../table/planner/plan/common/UnnestTestBase.scala |  129 +
 .../table/planner/plan/cost/FlinkCostTest.scala    |  187 ++
 .../metadata/AggCallSelectivityEstimatorTest.scala |  633 ++++
 .../metadata/FlinkRelMdColumnIntervalTest.scala    |  608 ++++
 .../metadata/FlinkRelMdColumnNullCountTest.scala   |  287 ++
 .../FlinkRelMdColumnOriginNullCountTest.scala      |  144 +
 .../metadata/FlinkRelMdColumnUniquenessTest.scala  |  610 ++++
 .../metadata/FlinkRelMdCumulativeCostTest.scala    |   52 +
 .../metadata/FlinkRelMdDistinctRowCountTest.scala  |  640 ++++
 .../plan/metadata/FlinkRelMdDistributionTest.scala |  109 +
 .../FlinkRelMdFilteredColumnIntervalTest.scala     |  187 ++
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  | 2308 ++++++++++++++
 .../FlinkRelMdModifiedMonotonicityTest.scala       |  290 ++
 .../metadata/FlinkRelMdNonCumulativeCostTest.scala |   51 +
 .../FlinkRelMdPercentageOriginalRowsTest.scala     |   88 +
 .../metadata/FlinkRelMdPopulationSizeTest.scala    |  355 +++
 .../plan/metadata/FlinkRelMdRowCountTest.scala     |  251 ++
 .../plan/metadata/FlinkRelMdSelectivityTest.scala  |  532 ++++
 .../planner/plan/metadata/FlinkRelMdSizeTest.scala |  209 ++
 .../plan/metadata/FlinkRelMdUniqueGroupsTest.scala |  643 ++++
 .../plan/metadata/FlinkRelMdUniqueKeysTest.scala   |  273 ++
 .../metadata/MetadataHandlerConsistencyTest.scala  |  150 +
 .../planner/plan/metadata/MetadataTestUtil.scala   |  270 ++
 .../plan/metadata/SelectivityEstimatorTest.scala   | 1071 +++++++
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  251 ++
 .../optimize/program/FlinkChainedProgramTest.scala |  157 +
 .../program/FlinkHepRuleSetProgramTest.scala       |  104 +
 .../optimize/program/FlinkVolcanoProgramTest.scala |   51 +
 .../logical/AggregateReduceGroupingRuleTest.scala  |   44 +
 .../logical/CalcPruneAggregateCallRuleTest.scala   |   50 +
 .../rules/logical/CalcRankTransposeRuleTest.scala  |  191 ++
 .../rules/logical/ConvertToNotInOrInRuleTest.scala |  168 ++
 .../logical/DecomposeGroupingSetsRuleTest.scala    |  145 +
 .../logical/ExpressionReductionRulesTest.scala     |   49 +
 ...AggregateExpandDistinctAggregatesRuleTest.scala |  174 ++
 .../FlinkAggregateJoinTransposeRuleTest.scala      |  147 +
 .../logical/FlinkAggregateRemoveRuleTest.scala     |  235 ++
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   85 +
 .../rules/logical/FlinkFilterJoinRuleTest.scala    |  158 +
 .../logical/FlinkJoinPushExpressionsRuleTest.scala |   80 +
 .../rules/logical/FlinkLimit0RemoveRuleTest.scala  |   99 +
 .../FlinkLogicalRankRuleForConstantRangeTest.scala |  211 ++
 .../FlinkLogicalRankRuleForRangeEndTest.scala      |  196 ++
 .../rules/logical/FlinkPruneEmptyRulesTest.scala   |   71 +
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.scala |   76 +
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.scala   |  359 +++
 ...FlinkSemiAntiJoinProjectTransposeRuleTest.scala |   81 +
 .../JoinConditionEqualityTransferRuleTest.scala    |  146 +
 .../logical/JoinConditionTypeCoerceRuleTest.scala  |  116 +
 .../JoinDependentConditionDerivationRuleTest.scala |  120 +
 .../logical/JoinDeriveNullFilterRuleTest.scala     |  119 +
 .../plan/rules/logical/LogicalUnnestRuleTest.scala |   50 +
 .../ProjectPruneAggregateCallRuleTest.scala        |   50 +
 .../ProjectSemiAntiJoinTransposeRuleTest.scala     |  151 +
 .../logical/PruneAggregateCallRuleTestBase.scala   |  176 ++
 .../PushFilterIntoTableSourceScanRuleTest.scala    |   97 +
 .../PushPartitionIntoTableSourceScanRuleTest.scala |   96 +
 .../PushProjectIntoTableSourceScanRuleTest.scala   |  127 +
 .../logical/RankNumberColumnRemoveRuleTest.scala   |   97 +
 .../ReplaceIntersectWithSemiJoinRuleTest.scala     |   74 +
 .../logical/ReplaceMinusWithAntiJoinRuleTest.scala |   74 +
 .../rules/logical/RewriteCoalesceRuleTest.scala    |  141 +
 .../logical/RewriteIntersectAllRuleTest.scala      |   75 +
 .../rules/logical/RewriteMinusAllRuleTest.scala    |   73 +
 .../RewriteMultiJoinConditionRuleTest.scala        |  151 +
 .../logical/SimplifyFilterConditionRuleTest.scala  |  108 +
 .../logical/SimplifyJoinConditionRuleTest.scala    |   68 +
 .../rules/logical/SplitAggregateRuleTest.scala     |  186 ++
 .../rules/logical/WindowGroupReorderRuleTest.scala |  180 ++
 .../subquery/FlinkRewriteSubQueryRuleTest.scala    |  212 ++
 .../logical/subquery/SubQueryAntiJoinTest.scala    |  769 +++++
 .../logical/subquery/SubQuerySemiJoinTest.scala    | 1674 ++++++++++
 .../rules/logical/subquery/SubQueryTestBase.scala  |   44 +
 .../SubqueryCorrelateVariablesValidationTest.scala |  130 +
 .../RemoveRedundantLocalHashAggRuleTest.scala      |   72 +
 .../batch/RemoveRedundantLocalRankRuleTest.scala   |   73 +
 .../RemoveRedundantLocalSortAggRuleTest.scala      |   67 +
 .../physical/stream/RetractionRulesTest.scala      |   70 +
 .../RetractionRulesWithTwoStageAggTest.scala       |   76 +
 .../plan/schema/TimeIndicatorRelDataTypeTest.scala |   42 +
 .../planner/plan/stats/ValueIntervalTest.scala     |  459 +++
 .../table/planner/plan/stream/sql/CalcTest.scala   |  159 +
 .../plan/stream/sql/DagOptimizationTest.scala      |  530 ++++
 .../table/planner/plan/stream/sql/LimitTest.scala  |  213 ++
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  352 +++
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  276 ++
 .../table/planner/plan/stream/sql/RankTest.scala   |  718 +++++
 .../stream/sql/RelTimeIndicatorConverterTest.scala |  178 ++
 .../planner/plan/stream/sql/SetOperatorsTest.scala |  128 +
 .../table/planner/plan/stream/sql/SinkTest.scala   |  153 +
 .../planner/plan/stream/sql/SortLimitTest.scala    |  327 ++
 .../table/planner/plan/stream/sql/SortTest.scala   |   79 +
 .../planner/plan/stream/sql/SubplanReuseTest.scala |  300 ++
 .../planner/plan/stream/sql/TableScanTest.scala    |   42 +
 .../planner/plan/stream/sql/TableSourceTest.scala  |  395 +++
 .../table/planner/plan/stream/sql/UnionTest.scala  |   66 +
 .../table/planner/plan/stream/sql/UnnestTest.scala |   26 +
 .../table/planner/plan/stream/sql/ValuesTest.scala |   49 +
 .../plan/stream/sql/agg/AggregateTest.scala        |  263 ++
 .../stream/sql/agg/DistinctAggregateTest.scala     |  224 ++
 .../plan/stream/sql/agg/GroupingSetsTest.scala     |  458 +++
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   52 +
 .../plan/stream/sql/agg/OverAggregateTest.scala    |  433 +++
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   79 +
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  298 ++
 .../plan/stream/sql/join/JoinReorderTest.scala     |   25 +
 .../planner/plan/stream/sql/join/JoinTest.scala    |  284 ++
 .../plan/stream/sql/join/LookupJoinTest.scala      |  493 +++
 .../plan/stream/sql/join/SemiAntiJoinTest.scala    |  576 ++++
 .../plan/stream/sql/join/TemporalJoinTest.scala    |  132 +
 .../plan/stream/sql/join/WindowJoinTest.scala      |  453 +++
 .../planner/plan/stream/table/AggregateTest.scala  |  230 ++
 .../table/planner/plan/stream/table/CalcTest.scala |  161 +
 .../plan/stream/table/ColumnFunctionsTest.scala    |  229 ++
 .../planner/plan/stream/table/CorrelateTest.scala  |  181 ++
 .../plan/stream/table/GroupWindowTest.scala        |  409 +++
 .../table/planner/plan/stream/table/JoinTest.scala |  263 ++
 .../planner/plan/stream/table/OverWindowTest.scala |  222 ++
 .../plan/stream/table/SetOperatorsTest.scala       |   87 +
 .../plan/stream/table/TableSourceTest.scala        |  302 ++
 .../plan/stream/table/TemporalTableJoinTest.scala  |  191 ++
 .../plan/stream/table/TwoStageAggregateTest.scala  |  112 +
 .../stringexpr/AggregateStringExpressionTest.scala |  246 ++
 .../stringexpr/CalcStringExpressionTest.scala      |  183 ++
 .../stringexpr/CorrelateStringExpressionTest.scala |  160 +
 .../GroupWindowStringExpressionTest.scala          |  263 ++
 ...pWindowTableAggregateStringExpressionTest.scala |  227 ++
 .../OverWindowStringExpressionTest.scala           |  248 ++
 .../SetOperatorsStringExpressionTest.scala         |   50 +
 .../table/validation/AggregateValidationTest.scala |  127 +
 .../table/validation/CalcValidationTest.scala      |  165 +
 .../table/validation/CorrelateValidationTest.scala |  178 ++
 .../validation/GroupWindowValidationTest.scala     |  308 ++
 .../validation/OverWindowValidationTest.scala      |  164 +
 .../validation/SetOperatorsValidationTest.scala    |   80 +
 .../table/validation/TableSinkValidationTest.scala |   87 +
 .../TemporalTableJoinValidationTest.scala          |  114 +
 .../validation/UnsupportedOpsValidationTest.scala  |  109 +
 .../plan/trait/FlinkRelDistributionTest.scala      |  165 +
 .../planner/plan/utils/FlinkRelOptUtilTest.scala   |  141 +
 .../planner/plan/utils/FlinkRexUtilTest.scala      |  407 +++
 .../planner/plan/utils/InputTypeBuilder.scala      |   53 +
 .../planner/plan/utils/PartitionPrunerTest.scala   |  111 +
 .../planner/plan/utils/RelDigestUtilTest.scala     |  108 +
 .../planner/plan/utils/RexNodeExtractorTest.scala  |  905 ++++++
 .../planner/plan/utils/RexNodeRewriterTest.scala   |   75 +
 .../table/planner/plan/utils/RexNodeTestBase.scala |   90 +
 .../flink/table/planner/plan/utils/pojos.scala     |   51 +
 .../planner/runtime/batch/sql/CalcITCase.scala     | 1263 ++++++++
 .../runtime/batch/sql/CorrelateITCase.scala        |  369 +++
 .../runtime/batch/sql/CorrelateITCase2.scala       |  267 ++
 .../planner/runtime/batch/sql/DecimalITCase.scala  |  936 ++++++
 .../runtime/batch/sql/Limit0RemoveITCase.scala     |   95 +
 .../planner/runtime/batch/sql/LimitITCase.scala    |  119 +
 .../planner/runtime/batch/sql/MiscITCase.scala     |  616 ++++
 .../runtime/batch/sql/OverWindowITCase.scala       | 2508 +++++++++++++++
 .../batch/sql/PartitionableSinkITCase.scala        |  330 ++
 .../planner/runtime/batch/sql/RankITCase.scala     |  141 +
 .../runtime/batch/sql/SetOperatorsITCase.scala     |  153 +
 .../runtime/batch/sql/SortLimitITCase.scala        |  129 +
 .../runtime/batch/sql/TableScanITCase.scala        |  124 +
 .../runtime/batch/sql/TableSourceITCase.scala      |  208 ++
 .../planner/runtime/batch/sql/UnionITCase.scala    |  140 +
 .../planner/runtime/batch/sql/UnnestITCase.scala   |  254 ++
 .../planner/runtime/batch/sql/ValuesITCase.scala   |   35 +
 .../batch/sql/agg/AggregateITCaseBase.scala        |  885 ++++++
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  208 ++
 .../sql/agg/AggregateReduceGroupingITCase.scala    |  413 +++
 .../batch/sql/agg/AggregateRemoveITCase.scala      |  213 ++
 .../sql/agg/DistinctAggregateITCaseBase.scala      |  312 ++
 .../runtime/batch/sql/agg/GroupingSetsITCase.scala |  564 ++++
 .../runtime/batch/sql/agg/HashAggITCase.scala      |   33 +
 .../sql/agg/HashDistinctAggregateITCase.scala      |   33 +
 .../batch/sql/agg/PruneAggregateCallITCase.scala   |  121 +
 .../runtime/batch/sql/agg/SortAggITCase.scala      |  568 ++++
 .../sql/agg/SortDistinctAggregateITCase.scala      |   97 +
 .../batch/sql/agg/WindowAggregateITCase.scala      |  744 +++++
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  199 ++
 .../sql/join/JoinConditionTypeCoerceITCase.scala   |  175 ++
 .../runtime/batch/sql/join/JoinITCase.scala        |  836 +++++
 .../runtime/batch/sql/join/JoinITCaseHelper.scala  |   53 +
 .../batch/sql/join/JoinWithoutKeyITCase.scala      |  373 +++
 .../runtime/batch/sql/join/LookupJoinITCase.scala  |  234 ++
 .../runtime/batch/sql/join/OuterJoinITCase.scala   |  394 +++
 .../runtime/batch/sql/join/ScalarQueryITCase.scala |   51 +
 .../runtime/batch/sql/join/SemiJoinITCase.scala    |  490 +++
 .../runtime/batch/table/AggregationITCase.scala    |  482 +++
 .../planner/runtime/batch/table/CalcITCase.scala   |  663 ++++
 .../runtime/batch/table/CorrelateITCase.scala      |  385 +++
 .../runtime/batch/table/DecimalITCase.scala        |  791 +++++
 .../runtime/batch/table/GroupWindowITCase.scala    |  342 +++
 .../planner/runtime/batch/table/JoinITCase.scala   |  391 +++
 .../runtime/batch/table/OverWindowITCase.scala     | 1404 +++++++++
 .../runtime/batch/table/SetOperatorsITCase.scala   |  198 ++
 .../planner/runtime/batch/table/SortITCase.scala   |   91 +
 ...AbstractTwoInputStreamOperatorWithTTLTest.scala |  182 ++
 .../planner/runtime/harness/HarnessTestBase.scala  |  122 +
 .../runtime/harness/OverWindowHarnessTest.scala    |  986 ++++++
 .../runtime/stream/sql/AggregateITCase.scala       | 1297 ++++++++
 .../runtime/stream/sql/AggregateRemoveITCase.scala |  254 ++
 .../runtime/stream/sql/AsyncLookupJoinITCase.scala |  352 +++
 .../planner/runtime/stream/sql/CalcITCase.scala    |  263 ++
 .../runtime/stream/sql/CorrelateITCase.scala       |  425 +++
 .../runtime/stream/sql/DeduplicateITCase.scala     |   89 +
 .../planner/runtime/stream/sql/JoinITCase.scala    | 1130 +++++++
 .../runtime/stream/sql/Limit0RemoveITCase.scala    |  187 ++
 .../planner/runtime/stream/sql/LimitITCase.scala   |  109 +
 .../runtime/stream/sql/LookupJoinITCase.scala      |  421 +++
 .../runtime/stream/sql/MatchRecognizeITCase.scala  |  783 +++++
 .../runtime/stream/sql/OverWindowITCase.scala      | 1023 +++++++
 .../stream/sql/PruneAggregateCallITCase.scala      |  130 +
 .../planner/runtime/stream/sql/RankITCase.scala    | 1314 ++++++++
 .../stream/sql/SemiAntiJoinStreamITCase.scala      |  542 ++++
 .../runtime/stream/sql/SetOperatorsITCase.scala    |  134 +
 .../planner/runtime/stream/sql/SortITCase.scala    |  240 ++
 .../runtime/stream/sql/SortLimitITCase.scala       |  105 +
 .../runtime/stream/sql/SplitAggregateITCase.scala  |  334 ++
 .../runtime/stream/sql/TableScanITCase.scala       |  147 +
 .../runtime/stream/sql/TableSourceITCase.scala     |  384 +++
 .../runtime/stream/sql/TemporalJoinITCase.scala    |  168 ++
 .../runtime/stream/sql/TemporalSortITCase.scala    |  163 +
 .../planner/runtime/stream/sql/UnnestITCase.scala  |  325 ++
 .../planner/runtime/stream/sql/ValuesITCase.scala  |   49 +
 .../runtime/stream/sql/WindowAggregateITCase.scala |  266 ++
 .../runtime/stream/sql/WindowJoinITCase.scala      |  948 ++++++
 .../runtime/stream/table/AggregateITCase.scala     |  401 +++
 .../planner/runtime/stream/table/CalcITCase.scala  |  467 +++
 .../runtime/stream/table/CorrelateITCase.scala     |  395 +++
 .../runtime/stream/table/GroupWindowITCase.scala   |  295 ++
 .../planner/runtime/stream/table/JoinITCase.scala  | 1472 +++++++++
 .../stream/table/MiniBatchGroupWindowITCase.scala  |  156 +
 .../runtime/stream/table/OverWindowITCase.scala    |  440 +++
 .../runtime/stream/table/RetractionITCase.scala    |  168 ++
 .../runtime/stream/table/SetOperatorsITCase.scala  |  205 ++
 .../runtime/stream/table/SubQueryITCase.scala      |  148 +
 .../planner/runtime/utils/BatchTableEnvUtil.scala  |  296 ++
 .../planner/runtime/utils/BatchTestBase.scala      |  490 +++
 .../runtime/utils/CollectionBatchExecTable.scala   |  403 +++
 .../utils/InMemoryLookupableTableSource.scala      |  273 ++
 .../planner/runtime/utils/SortTestUtils.scala      |   57 +
 .../planner/runtime/utils/StreamTableEnvUtil.scala |   53 +
 .../planner/runtime/utils/StreamTestSink.scala     |  524 ++++
 .../planner/runtime/utils/StreamingTestBase.scala  |   81 +
 .../runtime/utils/StreamingWithAggTestBase.scala   |   75 +
 .../utils/StreamingWithMiniBatchTestBase.scala     |   74 +
 .../runtime/utils/StreamingWithStateTestBase.scala |  244 ++
 .../table/planner/runtime/utils/TableUtil.scala    |   77 +
 .../table/planner/runtime/utils/TestData.scala     |  476 +++
 .../table/planner/runtime/utils/TestSinkUtil.scala |  121 +
 .../table/planner/runtime/utils/TimeTestUtil.scala |   68 +
 .../utils/UserDefinedFunctionTestUtils.scala       |  438 +++
 .../flink/table/planner/utils/AvgAggFunction.scala |  357 +++
 .../planner/utils/ColumnIntervalUtilTest.scala     |  218 ++
 .../table/planner/utils/CountAggFunction.scala     |   82 +
 .../table/planner/utils/DateTimeTestUtil.scala     |   41 +
 .../planner/utils/LogicalPlanFormatUtils.scala     |   40 +
 .../planner/utils/MemoryTableSourceSinkUtil.scala  |  166 +
 .../flink/table/planner/utils/SumAggFunction.scala |  184 ++
 .../flink/table/planner/utils/TableTestBase.scala  | 1074 +++++++
 .../planner/utils/UserDefinedAggFunctions.scala    |  187 ++
 .../planner/utils/UserDefinedTableFunctions.scala  |  520 ++++
 .../table/planner/utils/testTableSources.scala     |  599 ++++
 .../flink/table/runtime/batch/sql/CalcITCase.scala | 1262 --------
 .../table/runtime/batch/sql/CorrelateITCase.scala  |  369 ---
 .../table/runtime/batch/sql/CorrelateITCase2.scala |  267 --
 .../table/runtime/batch/sql/DecimalITCase.scala    |  935 ------
 .../runtime/batch/sql/Limit0RemoveITCase.scala     |   95 -
 .../table/runtime/batch/sql/LimitITCase.scala      |  119 -
 .../flink/table/runtime/batch/sql/MiscITCase.scala |  616 ----
 .../table/runtime/batch/sql/OverWindowITCase.scala | 2508 ---------------
 .../batch/sql/PartitionableSinkITCase.scala        |  328 --
 .../flink/table/runtime/batch/sql/RankITCase.scala |  142 -
 .../runtime/batch/sql/SetOperatorsITCase.scala     |  153 -
 .../table/runtime/batch/sql/SortLimitITCase.scala  |  129 -
 .../table/runtime/batch/sql/TableScanITCase.scala  |  124 -
 .../runtime/batch/sql/TableSourceITCase.scala      |  208 --
 .../table/runtime/batch/sql/UnionITCase.scala      |  140 -
 .../table/runtime/batch/sql/UnnestITCase.scala     |  256 --
 .../table/runtime/batch/sql/ValuesITCase.scala     |   35 -
 .../batch/sql/agg/AggregateITCaseBase.scala        |  885 ------
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  208 --
 .../sql/agg/AggregateReduceGroupingITCase.scala    |  412 ---
 .../batch/sql/agg/AggregateRemoveITCase.scala      |  213 --
 .../sql/agg/DistinctAggregateITCaseBase.scala      |  312 --
 .../runtime/batch/sql/agg/GroupingSetsITCase.scala |  564 ----
 .../runtime/batch/sql/agg/HashAggITCase.scala      |   34 -
 .../sql/agg/HashDistinctAggregateITCase.scala      |   33 -
 .../batch/sql/agg/PruneAggregateCallITCase.scala   |  121 -
 .../runtime/batch/sql/agg/SortAggITCase.scala      |  568 ----
 .../sql/agg/SortDistinctAggregateITCase.scala      |   97 -
 .../batch/sql/agg/WindowAggregateITCase.scala      |  744 -----
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  199 --
 .../sql/join/JoinConditionTypeCoerceITCase.scala   |  175 --
 .../table/runtime/batch/sql/join/JoinITCase.scala  |  835 -----
 .../runtime/batch/sql/join/JoinITCaseHelper.scala  |   52 -
 .../batch/sql/join/JoinWithoutKeyITCase.scala      |  373 ---
 .../runtime/batch/sql/join/LookupJoinITCase.scala  |  234 --
 .../runtime/batch/sql/join/OuterJoinITCase.scala   |  394 ---
 .../runtime/batch/sql/join/ScalarQueryITCase.scala |   51 -
 .../runtime/batch/sql/join/SemiJoinITCase.scala    |  490 ---
 .../runtime/batch/table/AggregationITCase.scala    |  482 ---
 .../table/runtime/batch/table/CalcITCase.scala     |  663 ----
 .../runtime/batch/table/CorrelateITCase.scala      |  385 ---
 .../table/runtime/batch/table/DecimalITCase.scala  |  790 -----
 .../runtime/batch/table/GroupWindowITCase.scala    |  342 ---
 .../table/runtime/batch/table/JoinITCase.scala     |  391 ---
 .../runtime/batch/table/OverWindowITCase.scala     | 1404 ---------
 .../runtime/batch/table/SetOperatorsITCase.scala   |  198 --
 .../table/runtime/batch/table/SortITCase.scala     |   91 -
 ...AbstractTwoInputStreamOperatorWithTTLTest.scala |  185 --
 .../table/runtime/harness/HarnessTestBase.scala    |  120 -
 .../runtime/harness/OverWindowHarnessTest.scala    |  986 ------
 .../table/runtime/stream/sql/AggregateITCase.scala | 1297 --------
 .../runtime/stream/sql/AggregateRemoveITCase.scala |  254 --
 .../runtime/stream/sql/AsyncLookupJoinITCase.scala |  351 ---
 .../table/runtime/stream/sql/CalcITCase.scala      |  263 --
 .../table/runtime/stream/sql/CorrelateITCase.scala |  425 ---
 .../runtime/stream/sql/DeduplicateITCase.scala     |   89 -
 .../table/runtime/stream/sql/JoinITCase.scala      | 1130 -------
 .../runtime/stream/sql/Limit0RemoveITCase.scala    |  187 --
 .../table/runtime/stream/sql/LimitITCase.scala     |  109 -
 .../runtime/stream/sql/LookupJoinITCase.scala      |  420 ---
 .../runtime/stream/sql/MatchRecognizeITCase.scala  |  782 -----
 .../runtime/stream/sql/OverWindowITCase.scala      | 1023 -------
 .../stream/sql/PruneAggregateCallITCase.scala      |  130 -
 .../table/runtime/stream/sql/RankITCase.scala      | 1314 --------
 .../stream/sql/SemiAntiJoinStreamITCase.scala      |  541 ----
 .../runtime/stream/sql/SetOperatorsITCase.scala    |  134 -
 .../table/runtime/stream/sql/SortITCase.scala      |  240 --
 .../table/runtime/stream/sql/SortLimitITCase.scala |  105 -
 .../runtime/stream/sql/SplitAggregateITCase.scala  |  334 --
 .../table/runtime/stream/sql/TableScanITCase.scala |  146 -
 .../runtime/stream/sql/TableSourceITCase.scala     |  385 ---
 .../runtime/stream/sql/TemporalJoinITCase.scala    |  168 --
 .../runtime/stream/sql/TemporalSortITCase.scala    |  163 -
 .../table/runtime/stream/sql/UnnestITCase.scala    |  325 --
 .../table/runtime/stream/sql/ValuesITCase.scala    |   49 -
 .../runtime/stream/sql/WindowAggregateITCase.scala |  266 --
 .../runtime/stream/sql/WindowJoinITCase.scala      |  948 ------
 .../runtime/stream/table/AggregateITCase.scala     |  401 ---
 .../table/runtime/stream/table/CalcITCase.scala    |  467 ---
 .../runtime/stream/table/CorrelateITCase.scala     |  395 ---
 .../runtime/stream/table/GroupWindowITCase.scala   |  295 --
 .../table/runtime/stream/table/JoinITCase.scala    | 1472 ---------
 .../stream/table/MiniBatchGroupWindowITCase.scala  |  156 -
 .../runtime/stream/table/OverWindowITCase.scala    |  439 ---
 .../runtime/stream/table/RetractionITCase.scala    |  168 --
 .../runtime/stream/table/SetOperatorsITCase.scala  |  205 --
 .../runtime/stream/table/SubQueryITCase.scala      |  148 -
 .../table/runtime/utils/BatchTableEnvUtil.scala    |  296 --
 .../flink/table/runtime/utils/BatchTestBase.scala  |  489 ---
 .../runtime/utils/CollectionBatchExecTable.scala   |  403 ---
 .../utils/InMemoryLookupableTableSource.scala      |  273 --
 .../flink/table/runtime/utils/SortTestUtils.scala  |   57 -
 .../table/runtime/utils/StreamTableEnvUtil.scala   |   53 -
 .../flink/table/runtime/utils/StreamTestSink.scala |  524 ----
 .../table/runtime/utils/StreamingTestBase.scala    |   81 -
 .../runtime/utils/StreamingWithAggTestBase.scala   |   74 -
 .../utils/StreamingWithMiniBatchTestBase.scala     |   73 -
 .../runtime/utils/StreamingWithStateTestBase.scala |  244 --
 .../flink/table/runtime/utils/TableUtil.scala      |   77 -
 .../flink/table/runtime/utils/TestData.scala       |  476 ---
 .../flink/table/runtime/utils/TestSinkUtil.scala   |  119 -
 .../flink/table/runtime/utils/TimeTestUtil.scala   |   68 -
 .../utils/UserDefinedFunctionTestUtils.scala       |  438 ---
 .../apache/flink/table/util/AvgAggFunction.scala   |  357 ---
 .../flink/table/util/ColumnIntervalUtilTest.scala  |  219 --
 .../apache/flink/table/util/CountAggFunction.scala |   82 -
 .../apache/flink/table/util/DateTimeTestUtil.scala |   41 -
 .../flink/table/util/LogicalPlanFormatUtils.scala  |   40 -
 .../table/util/MemoryTableSourceSinkUtil.scala     |  165 -
 .../apache/flink/table/util/SumAggFunction.scala   |  184 --
 .../apache/flink/table/util/TableTestBase.scala    | 1071 -------
 .../flink/table/util/UserDefinedAggFunctions.scala |  187 --
 .../table/util/UserDefinedTableFunctions.scala     |  520 ----
 .../apache/flink/table/util/testTableSources.scala |  599 ----
 .../expressions/PlannerExpressionParserImpl.scala  |    3 +
 flink-table/flink-table-runtime-blink/pom.xml      |    6 +
 .../flink/table/api/ExecutionConfigOptions.java    |  206 --
 .../apache/flink/table/api/window/CountWindow.java |  160 -
 .../apache/flink/table/api/window/TimeWindow.java  |  242 --
 .../org/apache/flink/table/api/window/Window.java  |   42 -
 .../table/dataformat/AbstractBinaryWriter.java     |    8 +-
 .../apache/flink/table/dataformat/BinaryArray.java |    2 +-
 .../flink/table/dataformat/BinaryArrayWriter.java  |    2 +-
 .../flink/table/dataformat/BinaryFormat.java       |    2 +-
 .../flink/table/dataformat/BinaryGeneric.java      |    2 +-
 .../apache/flink/table/dataformat/BinaryMap.java   |    4 +-
 .../apache/flink/table/dataformat/BinaryRow.java   |    2 +-
 .../flink/table/dataformat/BinaryRowWriter.java    |    2 +-
 .../flink/table/dataformat/BinaryString.java       |    4 +-
 .../flink/table/dataformat/BinaryStringUtil.java   |    2 +-
 .../flink/table/dataformat/BinaryWriter.java       |    6 +-
 .../table/dataformat/DataFormatConverters.java     |   10 +-
 .../org/apache/flink/table/dataformat/Decimal.java |    4 +-
 .../apache/flink/table/dataformat/NestedRow.java   |    2 +-
 .../flink/table/dataview/NullAwareMapIterator.java |   80 -
 .../table/dataview/PerKeyStateDataViewStore.java   |   77 -
 .../dataview/PerWindowStateDataViewStore.java      |   95 -
 .../apache/flink/table/dataview/StateDataView.java |   33 -
 .../flink/table/dataview/StateDataViewStore.java   |   59 -
 .../apache/flink/table/dataview/StateListView.java |  121 -
 .../apache/flink/table/dataview/StateMapView.java  |  402 ---
 .../flink/table/generated/AggsHandleFunction.java  |  104 -
 .../apache/flink/table/generated/CompileUtils.java |  104 -
 .../generated/GeneratedAggsHandleFunction.java     |   31 -
 .../flink/table/generated/GeneratedClass.java      |   98 -
 .../flink/table/generated/GeneratedCollector.java  |   42 -
 .../flink/table/generated/GeneratedFunction.java   |   42 -
 .../table/generated/GeneratedHashFunction.java     |   38 -
 .../flink/table/generated/GeneratedInput.java      |   42 -
 .../table/generated/GeneratedJoinCondition.java    |   38 -
 .../GeneratedNamespaceAggsHandleFunction.java      |   32 -
 .../generated/GeneratedNormalizedKeyComputer.java  |   37 -
 .../flink/table/generated/GeneratedOperator.java   |   42 -
 .../flink/table/generated/GeneratedProjection.java |   38 -
 .../table/generated/GeneratedRecordComparator.java |   39 -
 .../table/generated/GeneratedRecordEqualiser.java  |   38 -
 .../table/generated/GeneratedResultFuture.java     |   42 -
 .../apache/flink/table/generated/HashFunction.java |   31 -
 .../flink/table/generated/JoinCondition.java       |   34 -
 .../generated/NamespaceAggsHandleFunction.java     |   98 -
 .../table/generated/NormalizedKeyComputer.java     |   60 -
 .../apache/flink/table/generated/Projection.java   |   30 -
 .../flink/table/generated/RecordComparator.java    |   36 -
 .../flink/table/generated/RecordEqualiser.java     |   41 -
 .../runtime/AbstractProcessStreamOperator.java     |  111 -
 .../flink/table/runtime/BinaryHashPartitioner.java |   74 -
 .../table/runtime/CodeGenOperatorFactory.java      |   68 -
 .../flink/table/runtime/TableStreamOperator.java   |   49 -
 .../table/runtime/aggregate/BytesHashMap.java      |  707 -----
 .../BytesHashMapSpillMemorySegmentPool.java        |   61 -
 .../table/runtime/aggregate/GroupAggFunction.java  |  225 --
 .../aggregate/MiniBatchGlobalGroupAggFunction.java |  242 --
 .../aggregate/MiniBatchGroupAggFunction.java       |  251 --
 .../MiniBatchIncrementalGroupAggFunction.java      |  149 -
 .../aggregate/MiniBatchLocalGroupAggFunction.java  |  105 -
 .../table/runtime/aggregate/RecordCounter.java     |   92 -
 .../runtime/bundle/AbstractMapBundleOperator.java  |  175 --
 .../runtime/bundle/KeyedMapBundleOperator.java     |   41 -
 .../table/runtime/bundle/MapBundleFunction.java    |   62 -
 .../table/runtime/bundle/MapBundleOperator.java    |   47 -
 .../runtime/bundle/trigger/BundleTrigger.java      |   53 -
 .../bundle/trigger/BundleTriggerCallback.java      |   36 -
 .../runtime/bundle/trigger/CoBundleTrigger.java    |   62 -
 .../runtime/bundle/trigger/CountBundleTrigger.java |   62 -
 .../bundle/trigger/CountCoBundleTrigger.java       |   71 -
 .../runtime/dataview/NullAwareMapIterator.java     |   80 +
 .../runtime/dataview/PerKeyStateDataViewStore.java |   79 +
 .../dataview/PerWindowStateDataViewStore.java      |   97 +
 .../table/runtime/dataview/StateDataView.java      |   33 +
 .../table/runtime/dataview/StateDataViewStore.java |   61 +
 .../table/runtime/dataview/StateListView.java      |  121 +
 .../flink/table/runtime/dataview/StateMapView.java |  402 +++
 .../deduplicate/DeduplicateFunctionHelper.java     |   81 -
 .../DeduplicateKeepFirstRowFunction.java           |   68 -
 .../DeduplicateKeepLastRowFunction.java            |   81 -
 .../MiniBatchDeduplicateKeepFirstRowFunction.java  |   81 -
 .../MiniBatchDeduplicateKeepLastRowFunction.java   |   81 -
 .../runtime/functions/SqlLikeChainChecker.java     |    2 +-
 .../runtime/generated/AggsHandleFunction.java      |  104 +
 .../table/runtime/generated/CompileUtils.java      |  104 +
 .../generated/GeneratedAggsHandleFunction.java     |   31 +
 .../table/runtime/generated/GeneratedClass.java    |   98 +
 .../runtime/generated/GeneratedCollector.java      |   42 +
 .../table/runtime/generated/GeneratedFunction.java |   42 +
 .../runtime/generated/GeneratedHashFunction.java   |   38 +
 .../table/runtime/generated/GeneratedInput.java    |   42 +
 .../runtime/generated/GeneratedJoinCondition.java  |   38 +
 .../GeneratedNamespaceAggsHandleFunction.java      |   32 +
 .../generated/GeneratedNormalizedKeyComputer.java  |   37 +
 .../table/runtime/generated/GeneratedOperator.java |   42 +
 .../runtime/generated/GeneratedProjection.java     |   38 +
 .../generated/GeneratedRecordComparator.java       |   39 +
 .../generated/GeneratedRecordEqualiser.java        |   38 +
 .../runtime/generated/GeneratedResultFuture.java   |   42 +
 .../table/runtime/generated/HashFunction.java      |   31 +
 .../table/runtime/generated/JoinCondition.java     |   34 +
 .../generated/NamespaceAggsHandleFunction.java     |   98 +
 .../runtime/generated/NormalizedKeyComputer.java   |   60 +
 .../flink/table/runtime/generated/Projection.java  |   30 +
 .../table/runtime/generated/RecordComparator.java  |   36 +
 .../table/runtime/generated/RecordEqualiser.java   |   41 +
 .../runtime/hashtable/BaseHybridHashTable.java     |    2 +-
 .../runtime/hashtable/BinaryHashPartition.java     |    2 +-
 .../table/runtime/hashtable/BinaryHashTable.java   |   12 +-
 .../table/runtime/hashtable/BuildSideIterator.java |    2 +-
 .../table/runtime/hashtable/LongHashPartition.java |    2 +-
 .../runtime/hashtable/LongHybridHashTable.java     |    2 +-
 .../io/BinaryRowChannelInputViewIterator.java      |    2 +-
 .../table/runtime/join/EmitAwareCollector.java     |   56 -
 .../flink/table/runtime/join/FlinkJoinType.java    |   77 -
 .../flink/table/runtime/join/HashJoinOperator.java |  455 ---
 .../flink/table/runtime/join/HashJoinType.java     |   79 -
 .../KeyedCoProcessOperatorWithWatermarkDelay.java  |   61 -
 .../table/runtime/join/NullAwareJoinHelper.java    |   51 -
 .../table/runtime/join/OuterJoinPaddingUtil.java   |   91 -
 .../runtime/join/ProcTimeBoundedStreamJoin.java    |   63 -
 .../runtime/join/RowTimeBoundedStreamJoin.java     |   83 -
 .../join/SortMergeFullOuterJoinIterator.java       |  187 --
 .../runtime/join/SortMergeInnerJoinIterator.java   |   79 -
 .../table/runtime/join/SortMergeJoinIterator.java  |  141 -
 .../table/runtime/join/SortMergeJoinOperator.java  |  457 ---
 .../join/SortMergeOneSideOuterJoinIterator.java    |   88 -
 .../table/runtime/join/TimeBoundedStreamJoin.java  |  477 ---
 .../runtime/join/lookup/AsyncLookupJoinRunner.java |  293 --
 .../join/lookup/AsyncLookupJoinWithCalcRunner.java |  152 -
 .../join/lookup/DelegatingResultFuture.java        |   55 -
 .../runtime/join/lookup/LookupJoinRunner.java      |  105 -
 .../join/lookup/LookupJoinWithCalcRunner.java      |   93 -
 .../join/stream/AbstractStreamingJoinOperator.java |  272 --
 .../runtime/join/stream/StreamingJoinOperator.java |  300 --
 .../join/stream/StreamingSemiAntiJoinOperator.java |  199 --
 .../join/stream/state/JoinInputSideSpec.java       |  128 -
 .../join/stream/state/JoinRecordStateView.java     |   47 -
 .../join/stream/state/JoinRecordStateViews.java    |  253 --
 .../stream/state/OuterJoinRecordStateView.java     |   56 -
 .../stream/state/OuterJoinRecordStateViews.java    |  329 --
 ...seTwoInputStreamOperatorWithStateRetention.java |  162 -
 .../temporal/TemporalProcessTimeJoinOperator.java  |  126 -
 .../join/temporal/TemporalRowTimeJoinOperator.java |  400 ---
 .../runtime/keyselector/BaseRowKeySelector.java    |    2 +-
 .../runtime/keyselector/BinaryRowKeySelector.java  |    6 +-
 .../keyselector/NullBinaryRowKeySelector.java      |    2 +-
 .../runtime/match/BaseRowEventComparator.java      |   48 -
 .../runtime/match/IterativeConditionRunner.java    |   57 -
 .../match/PatternProcessFunctionRunner.java        |   61 -
 .../runtime/match/RowtimeProcessFunction.java      |   56 -
 .../operators/AbstractProcessStreamOperator.java   |  111 +
 .../runtime/operators/CodeGenOperatorFactory.java  |   69 +
 .../runtime/operators/TableStreamOperator.java     |   49 +
 .../runtime/operators/aggregate/BytesHashMap.java  |  707 +++++
 .../BytesHashMapSpillMemorySegmentPool.java        |   61 +
 .../operators/aggregate/GroupAggFunction.java      |  225 ++
 .../aggregate/MiniBatchGlobalGroupAggFunction.java |  242 ++
 .../aggregate/MiniBatchGroupAggFunction.java       |  251 ++
 .../MiniBatchIncrementalGroupAggFunction.java      |  149 +
 .../aggregate/MiniBatchLocalGroupAggFunction.java  |  105 +
 .../runtime/operators/aggregate/RecordCounter.java |   92 +
 .../bundle/AbstractMapBundleOperator.java          |  175 ++
 .../operators/bundle/KeyedMapBundleOperator.java   |   41 +
 .../operators/bundle/MapBundleFunction.java        |   62 +
 .../operators/bundle/MapBundleOperator.java        |   47 +
 .../operators/bundle/trigger/BundleTrigger.java    |   53 +
 .../bundle/trigger/BundleTriggerCallback.java      |   36 +
 .../operators/bundle/trigger/CoBundleTrigger.java  |   62 +
 .../bundle/trigger/CountBundleTrigger.java         |   62 +
 .../bundle/trigger/CountCoBundleTrigger.java       |   71 +
 .../deduplicate/DeduplicateFunctionHelper.java     |   81 +
 .../DeduplicateKeepFirstRowFunction.java           |   68 +
 .../DeduplicateKeepLastRowFunction.java            |   81 +
 .../MiniBatchDeduplicateKeepFirstRowFunction.java  |   81 +
 .../MiniBatchDeduplicateKeepLastRowFunction.java   |   81 +
 .../runtime/operators/join/EmitAwareCollector.java |   56 +
 .../runtime/operators/join/FlinkJoinType.java      |   77 +
 .../runtime/operators/join/HashJoinOperator.java   |  455 +++
 .../table/runtime/operators/join/HashJoinType.java |   79 +
 .../KeyedCoProcessOperatorWithWatermarkDelay.java  |   61 +
 .../operators/join/NullAwareJoinHelper.java        |   51 +
 .../operators/join/OuterJoinPaddingUtil.java       |   91 +
 .../operators/join/ProcTimeBoundedStreamJoin.java  |   63 +
 .../operators/join/RowTimeBoundedStreamJoin.java   |   83 +
 .../join/SortMergeFullOuterJoinIterator.java       |  187 ++
 .../operators/join/SortMergeInnerJoinIterator.java |   79 +
 .../operators/join/SortMergeJoinIterator.java      |  141 +
 .../operators/join/SortMergeJoinOperator.java      |  457 +++
 .../join/SortMergeOneSideOuterJoinIterator.java    |   88 +
 .../operators/join/TimeBoundedStreamJoin.java      |  477 +++
 .../join/lookup/AsyncLookupJoinRunner.java         |  293 ++
 .../join/lookup/AsyncLookupJoinWithCalcRunner.java |  152 +
 .../join/lookup/DelegatingResultFuture.java        |   55 +
 .../operators/join/lookup/LookupJoinRunner.java    |  105 +
 .../join/lookup/LookupJoinWithCalcRunner.java      |   93 +
 .../join/stream/AbstractStreamingJoinOperator.java |  272 ++
 .../join/stream/StreamingJoinOperator.java         |  300 ++
 .../join/stream/StreamingSemiAntiJoinOperator.java |  199 ++
 .../join/stream/state/JoinInputSideSpec.java       |  128 +
 .../join/stream/state/JoinRecordStateView.java     |   47 +
 .../join/stream/state/JoinRecordStateViews.java    |  253 ++
 .../stream/state/OuterJoinRecordStateView.java     |   56 +
 .../stream/state/OuterJoinRecordStateViews.java    |  329 ++
 ...seTwoInputStreamOperatorWithStateRetention.java |  162 +
 .../temporal/TemporalProcessTimeJoinOperator.java  |  126 +
 .../join/temporal/TemporalRowTimeJoinOperator.java |  400 +++
 .../operators/match/BaseRowEventComparator.java    |   48 +
 .../operators/match/IterativeConditionRunner.java  |   57 +
 .../match/PatternProcessFunctionRunner.java        |   61 +
 .../operators/match/RowtimeProcessFunction.java    |   56 +
 .../AbstractRowTimeUnboundedPrecedingOver.java     |  266 ++
 .../over/BufferDataOverWindowOperator.java         |  140 +
 .../over/NonBufferOverWindowOperator.java          |  112 +
 .../ProcTimeRangeBoundedPrecedingFunction.java     |  246 ++
 .../over/ProcTimeRowsBoundedPrecedingFunction.java |  235 ++
 .../over/ProcTimeUnboundedPrecedingFunction.java   |  126 +
 .../over/RowTimeRangeBoundedPrecedingFunction.java |  289 ++
 .../RowTimeRangeUnboundedPrecedingFunction.java    |   79 +
 .../over/RowTimeRowsBoundedPrecedingFunction.java  |  301 ++
 .../RowTimeRowsUnboundedPrecedingFunction.java     |   71 +
 .../operators/over/frame/InsensitiveOverFrame.java |   61 +
 .../operators/over/frame/OffsetOverFrame.java      |  128 +
 .../operators/over/frame/OverWindowFrame.java      |   75 +
 .../over/frame/RangeSlidingOverFrame.java          |   93 +
 .../frame/RangeUnboundedFollowingOverFrame.java    |   74 +
 .../frame/RangeUnboundedPrecedingOverFrame.java    |   69 +
 .../operators/over/frame/RowSlidingOverFrame.java  |   92 +
 .../over/frame/RowUnboundedFollowingOverFrame.java |   62 +
 .../over/frame/RowUnboundedPrecedingOverFrame.java |   69 +
 .../operators/over/frame/SlidingOverFrame.java     |  104 +
 .../over/frame/UnboundedFollowingOverFrame.java    |   99 +
 .../over/frame/UnboundedOverWindowFrame.java       |   80 +
 .../over/frame/UnboundedPrecedingOverFrame.java    |   72 +
 .../operators/rank/AbstractTopNFunction.java       |  317 ++
 .../operators/rank/AppendOnlyTopNFunction.java     |  222 ++
 .../runtime/operators/rank/ConstantRankRange.java  |   53 +
 .../rank/ConstantRankRangeWithoutEnd.java          |   43 +
 .../table/runtime/operators/rank/RankRange.java    |   32 +
 .../table/runtime/operators/rank/RankType.java     |   49 +
 .../operators/rank/RetractableTopNFunction.java    |  307 ++
 .../table/runtime/operators/rank/TopNBuffer.java   |  221 ++
 .../operators/rank/UpdatableTopNFunction.java      |  493 +++
 .../runtime/operators/rank/VariableRankRange.java  |   49 +
 .../sort/AbstractBinaryExternalMerger.java         |  224 ++
 .../operators/sort/BaseTemporalSortOperator.java   |   50 +
 .../operators/sort/BinaryExternalMerger.java       |   88 +
 .../operators/sort/BinaryExternalSorter.java       | 1169 +++++++
 .../operators/sort/BinaryInMemorySortBuffer.java   |  224 ++
 .../operators/sort/BinaryIndexedSortable.java      |  251 ++
 .../operators/sort/BinaryKVExternalMerger.java     |   95 +
 .../operators/sort/BinaryKVInMemorySortBuffer.java |  153 +
 .../operators/sort/BinaryMergeIterator.java        |   99 +
 .../operators/sort/BufferedKVExternalSorter.java   |  186 ++
 .../sort/ChannelReaderKVInputViewIterator.java     |   70 +
 .../runtime/operators/sort/LimitOperator.java      |   63 +
 .../operators/sort/ListMemorySegmentPool.java      |   61 +
 .../operators/sort/ProcTimeSortOperator.java       |  114 +
 .../table/runtime/operators/sort/RankOperator.java |  112 +
 .../operators/sort/RowTimeSortOperator.java        |  150 +
 .../runtime/operators/sort/SortLimitOperator.java  |  106 +
 .../table/runtime/operators/sort/SortOperator.java |  113 +
 .../table/runtime/operators/sort/SortUtil.java     |  213 ++
 .../operators/sort/SpillChannelManager.java        |  101 +
 .../runtime/operators/sort/StreamSortOperator.java |  160 +
 .../operators/values/ValuesInputFormat.java        |   76 +
 .../runtime/operators/window/CountWindow.java      |  160 +
 .../table/runtime/operators/window/TimeWindow.java |  242 ++
 .../table/runtime/operators/window/Window.java     |   42 +
 .../runtime/operators/window/WindowOperator.java   |  706 +++++
 .../operators/window/WindowOperatorBuilder.java    |  231 ++
 .../assigners/CountSlidingWindowAssigner.java      |  104 +
 .../assigners/CountTumblingWindowAssigner.java     |   85 +
 .../assigners/InternalTimeWindowAssigner.java      |   35 +
 .../window/assigners/MergingWindowAssigner.java    |   60 +
 .../window/assigners/PanedWindowAssigner.java      |   51 +
 .../window/assigners/SessionWindowAssigner.java    |  127 +
 .../window/assigners/SlidingWindowAssigner.java    |  170 ++
 .../window/assigners/TumblingWindowAssigner.java   |  123 +
 .../operators/window/assigners/WindowAssigner.java |   76 +
 .../window/grouping/HeapWindowsGrouping.java       |  113 +
 .../operators/window/grouping/WindowsGrouping.java |  277 ++
 .../internal/GeneralWindowProcessFunction.java     |   85 +
 .../internal/InternalWindowProcessFunction.java    |  209 ++
 .../internal/MergingWindowProcessFunction.java     |  176 ++
 .../window/internal/MergingWindowSet.java          |  258 ++
 .../internal/PanedWindowProcessFunction.java       |  107 +
 .../operators/window/triggers/ElementTriggers.java |  177 ++
 .../window/triggers/EventTimeTriggers.java         |  354 +++
 .../window/triggers/ProcessingTimeTriggers.java    |  297 ++
 .../runtime/operators/window/triggers/Trigger.java |  201 ++
 .../wmassigners/MiniBatchAssignerOperator.java     |  105 +
 .../MiniBatchedWatermarkAssignerOperator.java      |  171 ++
 .../wmassigners/WatermarkAssignerOperator.java     |  170 ++
 .../AbstractRowTimeUnboundedPrecedingOver.java     |  266 --
 .../runtime/over/BufferDataOverWindowOperator.java |  140 -
 .../runtime/over/NonBufferOverWindowOperator.java  |  112 -
 .../ProcTimeRangeBoundedPrecedingFunction.java     |  246 --
 .../over/ProcTimeRowsBoundedPrecedingFunction.java |  235 --
 .../over/ProcTimeUnboundedPrecedingFunction.java   |  126 -
 .../over/RowTimeRangeBoundedPrecedingFunction.java |  289 --
 .../RowTimeRangeUnboundedPrecedingFunction.java    |   79 -
 .../over/RowTimeRowsBoundedPrecedingFunction.java  |  301 --
 .../RowTimeRowsUnboundedPrecedingFunction.java     |   71 -
 .../runtime/over/frame/InsensitiveOverFrame.java   |   61 -
 .../table/runtime/over/frame/OffsetOverFrame.java  |  128 -
 .../table/runtime/over/frame/OverWindowFrame.java  |   75 -
 .../runtime/over/frame/RangeSlidingOverFrame.java  |   93 -
 .../frame/RangeUnboundedFollowingOverFrame.java    |   74 -
 .../frame/RangeUnboundedPrecedingOverFrame.java    |   69 -
 .../runtime/over/frame/RowSlidingOverFrame.java    |   92 -
 .../over/frame/RowUnboundedFollowingOverFrame.java |   62 -
 .../over/frame/RowUnboundedPrecedingOverFrame.java |   69 -
 .../table/runtime/over/frame/SlidingOverFrame.java |  104 -
 .../over/frame/UnboundedFollowingOverFrame.java    |   99 -
 .../over/frame/UnboundedOverWindowFrame.java       |   80 -
 .../over/frame/UnboundedPrecedingOverFrame.java    |   72 -
 .../runtime/partitioner/BinaryHashPartitioner.java |   74 +
 .../table/runtime/rank/AbstractTopNFunction.java   |  317 --
 .../table/runtime/rank/AppendOnlyTopNFunction.java |  222 --
 .../table/runtime/rank/ConstantRankRange.java      |   53 -
 .../runtime/rank/ConstantRankRangeWithoutEnd.java  |   43 -
 .../apache/flink/table/runtime/rank/RankRange.java |   32 -
 .../apache/flink/table/runtime/rank/RankType.java  |   49 -
 .../runtime/rank/RetractableTopNFunction.java      |  307 --
 .../flink/table/runtime/rank/TopNBuffer.java       |  221 --
 .../table/runtime/rank/UpdatableTopNFunction.java  |  493 ---
 .../table/runtime/rank/VariableRankRange.java      |   49 -
 .../runtime/sort/AbstractBinaryExternalMerger.java |  224 --
 .../runtime/sort/BaseTemporalSortOperator.java     |   50 -
 .../table/runtime/sort/BinaryExternalMerger.java   |   88 -
 .../table/runtime/sort/BinaryExternalSorter.java   | 1169 -------
 .../runtime/sort/BinaryInMemorySortBuffer.java     |  224 --
 .../table/runtime/sort/BinaryIndexedSortable.java  |  251 --
 .../table/runtime/sort/BinaryKVExternalMerger.java |   95 -
 .../runtime/sort/BinaryKVInMemorySortBuffer.java   |  153 -
 .../table/runtime/sort/BinaryMergeIterator.java    |   99 -
 .../runtime/sort/BufferedKVExternalSorter.java     |  186 --
 .../sort/ChannelReaderKVInputViewIterator.java     |   70 -
 .../flink/table/runtime/sort/LimitOperator.java    |   63 -
 .../table/runtime/sort/ListMemorySegmentPool.java  |   61 -
 .../table/runtime/sort/ProcTimeSortOperator.java   |  114 -
 .../flink/table/runtime/sort/RankOperator.java     |  112 -
 .../table/runtime/sort/RowTimeSortOperator.java    |  150 -
 .../table/runtime/sort/SortLimitOperator.java      |  106 -
 .../flink/table/runtime/sort/SortOperator.java     |  113 -
 .../apache/flink/table/runtime/sort/SortUtil.java  |  213 --
 .../table/runtime/sort/SpillChannelManager.java    |  101 -
 .../table/runtime/sort/StreamSortOperator.java     |  160 -
 .../runtime/types/ClassDataTypeConverter.java      |   47 +
 .../runtime/types/ClassLogicalTypeConverter.java   |   86 +
 .../table/runtime/types/InternalSerializers.java   |   99 +
 .../types/LogicalTypeDataTypeConverter.java        |  142 +
 .../table/runtime/types/PlannerTypeUtils.java      |  113 +
 .../runtime/types/TypeInfoDataTypeConverter.java   |  148 +
 .../types/TypeInfoLogicalTypeConverter.java        |   60 +
 .../runtime/typeutils/AbstractMapSerializer.java   |  194 ++
 .../runtime/typeutils/AbstractMapTypeInfo.java     |  140 +
 .../runtime/typeutils/AbstractRowSerializer.java   |  106 +
 .../runtime/typeutils/BaseArraySerializer.java     |  293 ++
 .../table/runtime/typeutils/BaseMapSerializer.java |  321 ++
 .../table/runtime/typeutils/BaseRowSerializer.java |  351 +++
 .../table/runtime/typeutils/BaseRowTypeInfo.java   |  212 ++
 .../runtime/typeutils/BigDecimalTypeInfo.java      |   91 +
 .../runtime/typeutils/BinaryGenericSerializer.java |  189 ++
 .../runtime/typeutils/BinaryRowSerializer.java     |  396 +++
 .../runtime/typeutils/BinaryStringSerializer.java  |  115 +
 .../runtime/typeutils/BinaryStringTypeInfo.java    |   89 +
 .../typeutils/BinaryStringTypeInfoFactory.java     |   39 +
 .../table/runtime/typeutils/DecimalSerializer.java |  201 ++
 .../table/runtime/typeutils/DecimalTypeInfo.java   |  113 +
 .../runtime/typeutils/DecimalTypeInfoFactory.java  |   37 +
 .../runtime/typeutils/SortedMapSerializer.java     |  119 +
 .../typeutils/SortedMapSerializerSnapshot.java     |  110 +
 .../table/runtime/typeutils/SortedMapTypeInfo.java |  145 +
 .../table/runtime/typeutils/TypeCheckUtils.java    |  160 +
 .../flink/table/runtime/util/MurmurHashUtil.java   |  169 ++
 .../runtime/util/ResettableExternalBuffer.java     |    4 +-
 .../flink/table/runtime/util/SegmentsUtil.java     | 1032 +++++++
 .../flink/table/runtime/util/StringUtf8Utils.java  |    4 +-
 .../runtime/util/collections/DoubleHashSet.java    |    2 +-
 .../runtime/util/collections/FloatHashSet.java     |    2 +-
 .../table/runtime/util/collections/IntHashSet.java |    2 +-
 .../runtime/util/collections/LongHashSet.java      |    2 +-
 .../runtime/util/collections/ShortHashSet.java     |    2 +-
 .../table/runtime/values/ValuesInputFormat.java    |   76 -
 .../MiniBatchAssignerOperator.java                 |  105 -
 .../MiniBatchedWatermarkAssignerOperator.java      |  171 --
 .../WatermarkAssignerOperator.java                 |  170 --
 .../flink/table/runtime/window/WindowOperator.java |  707 -----
 .../runtime/window/WindowOperatorBuilder.java      |  231 --
 .../assigners/CountSlidingWindowAssigner.java      |  104 -
 .../assigners/CountTumblingWindowAssigner.java     |   85 -
 .../assigners/InternalTimeWindowAssigner.java      |   35 -
 .../window/assigners/MergingWindowAssigner.java    |   60 -
 .../window/assigners/PanedWindowAssigner.java      |   51 -
 .../window/assigners/SessionWindowAssigner.java    |  127 -
 .../window/assigners/SlidingWindowAssigner.java    |  170 --
 .../window/assigners/TumblingWindowAssigner.java   |  123 -
 .../runtime/window/assigners/WindowAssigner.java   |   76 -
 .../window/grouping/HeapWindowsGrouping.java       |  113 -
 .../runtime/window/grouping/WindowsGrouping.java   |  277 --
 .../internal/GeneralWindowProcessFunction.java     |   85 -
 .../internal/InternalWindowProcessFunction.java    |  209 --
 .../internal/MergingWindowProcessFunction.java     |  176 --
 .../runtime/window/internal/MergingWindowSet.java  |  258 --
 .../internal/PanedWindowProcessFunction.java       |  107 -
 .../runtime/window/triggers/ElementTriggers.java   |  177 --
 .../runtime/window/triggers/EventTimeTriggers.java |  354 ---
 .../window/triggers/ProcessingTimeTriggers.java    |  297 --
 .../table/runtime/window/triggers/Trigger.java     |  201 --
 .../flink/table/types/ClassDataTypeConverter.java  |   46 -
 .../table/types/ClassLogicalTypeConverter.java     |   86 -
 .../flink/table/types/InternalSerializers.java     |   99 -
 .../table/types/LogicalTypeDataTypeConverter.java  |  141 -
 .../apache/flink/table/types/PlannerTypeUtils.java |  112 -
 .../table/types/TypeInfoDataTypeConverter.java     |  144 -
 .../table/types/TypeInfoLogicalTypeConverter.java  |   59 -
 .../table/typeutils/AbstractMapSerializer.java     |  194 --
 .../flink/table/typeutils/AbstractMapTypeInfo.java |  140 -
 .../table/typeutils/AbstractRowSerializer.java     |  106 -
 .../flink/table/typeutils/BaseArraySerializer.java |  292 --
 .../flink/table/typeutils/BaseMapSerializer.java   |  320 --
 .../flink/table/typeutils/BaseRowSerializer.java   |  351 ---
 .../flink/table/typeutils/BaseRowTypeInfo.java     |  212 --
 .../flink/table/typeutils/BigDecimalTypeInfo.java  |   91 -
 .../table/typeutils/BinaryGenericSerializer.java   |  189 --
 .../flink/table/typeutils/BinaryRowSerializer.java |  396 ---
 .../table/typeutils/BinaryStringSerializer.java    |  115 -
 .../table/typeutils/BinaryStringTypeInfo.java      |   89 -
 .../typeutils/BinaryStringTypeInfoFactory.java     |   39 -
 .../flink/table/typeutils/DecimalSerializer.java   |  201 --
 .../flink/table/typeutils/DecimalTypeInfo.java     |  113 -
 .../table/typeutils/DecimalTypeInfoFactory.java    |   37 -
 .../flink/table/typeutils/SortedMapSerializer.java |  119 -
 .../typeutils/SortedMapSerializerSnapshot.java     |  110 -
 .../flink/table/typeutils/SortedMapTypeInfo.java   |  145 -
 .../flink/table/typeutils/TypeCheckUtils.java      |  160 -
 .../apache/flink/table/util/MurmurHashUtil.java    |  168 --
 .../org/apache/flink/table/util/SegmentsUtil.java  | 1031 -------
 .../apache/flink/table/dataformat/BaseRowTest.java |    6 +-
 .../flink/table/dataformat/BinaryArrayTest.java    |    8 +-
 .../flink/table/dataformat/BinaryRowTest.java      |    8 +-
 .../flink/table/dataformat/BinaryStringTest.java   |    2 +-
 .../table/dataformat/DataFormatConvertersTest.java |    6 +-
 .../flink/table/dataformat/NestedRowTest.java      |    2 +-
 .../flink/table/generated/CompileUtilsTest.java    |   82 -
 .../table/generated/GeneratedCollectorWrapper.java |   51 -
 .../table/generated/GeneratedFunctionWrapper.java  |   51 -
 .../generated/GeneratedResultFutureWrapper.java    |   51 -
 .../table/runtime/aggregate/BytesHashMapTest.java  |  491 ---
 .../flink/table/runtime/aggregate/HashAggTest.java |  152 -
 .../runtime/aggregate/SumHashAggTestOperator.java  |  259 --
 .../runtime/bundle/MapBundleOperatorTest.java      |  120 -
 .../bundle/trigger/CountBundleTriggerTest.java     |   48 -
 .../bundle/trigger/CountCoBundleTriggerTest.java   |   54 -
 .../bundle/trigger/TestTriggerCallback.java        |   36 -
 .../deduplicate/DeduplicateFunctionTestBase.java   |   49 -
 .../DeduplicateKeepFirstRowFunctionTest.java       |   63 -
 .../DeduplicateKeepLastRowFunctionTest.java        |   86 -
 ...niBatchDeduplicateKeepFirstRowFunctionTest.java |   74 -
 ...iniBatchDeduplicateKeepLastRowFunctionTest.java |  116 -
 .../table/runtime/generated/CompileUtilsTest.java  |   82 +
 .../generated/GeneratedCollectorWrapper.java       |   51 +
 .../generated/GeneratedFunctionWrapper.java        |   51 +
 .../generated/GeneratedResultFutureWrapper.java    |   51 +
 .../runtime/hashtable/BinaryHashTableTest.java     |    8 +-
 .../table/runtime/hashtable/LongHashTableTest.java |    4 +-
 .../runtime/join/AsyncLookupJoinHarnessTest.java   |  348 ---
 .../runtime/join/Int2HashJoinOperatorTest.java     |  438 ---
 .../join/Int2SortMergeJoinOperatorTest.java        |  237 --
 .../table/runtime/join/LookupJoinHarnessTest.java  |  281 --
 .../join/ProcTimeBoundedStreamJoinTest.java        |  178 --
 .../runtime/join/RandomSortMergeInnerJoinTest.java |  393 ---
 .../runtime/join/RandomSortMergeOuterJoinTest.java |  224 --
 .../runtime/join/RowTimeBoundedStreamJoinTest.java |  387 ---
 .../runtime/join/SortMergeJoinIteratorTest.java    |  387 ---
 .../runtime/join/String2HashJoinOperatorTest.java  |  318 --
 .../join/String2SortMergeJoinOperatorTest.java     |  220 --
 .../join/TimeBoundedStreamJoinTestBase.java        |   64 -
 .../operators/aggregate/BytesHashMapTest.java      |  491 +++
 .../runtime/operators/aggregate/HashAggTest.java   |  152 +
 .../aggregate/SumHashAggTestOperator.java          |  259 ++
 .../operators/bundle/MapBundleOperatorTest.java    |  120 +
 .../bundle/trigger/CountBundleTriggerTest.java     |   48 +
 .../bundle/trigger/CountCoBundleTriggerTest.java   |   54 +
 .../bundle/trigger/TestTriggerCallback.java        |   36 +
 .../deduplicate/DeduplicateFunctionTestBase.java   |   49 +
 .../DeduplicateKeepFirstRowFunctionTest.java       |   63 +
 .../DeduplicateKeepLastRowFunctionTest.java        |   86 +
 ...niBatchDeduplicateKeepFirstRowFunctionTest.java |   74 +
 ...iniBatchDeduplicateKeepLastRowFunctionTest.java |  116 +
 .../operators/join/AsyncLookupJoinHarnessTest.java |  348 +++
 .../operators/join/Int2HashJoinOperatorTest.java   |  438 +++
 .../join/Int2SortMergeJoinOperatorTest.java        |  237 ++
 .../operators/join/LookupJoinHarnessTest.java      |  281 ++
 .../join/ProcTimeBoundedStreamJoinTest.java        |  178 ++
 .../join/RandomSortMergeInnerJoinTest.java         |  393 +++
 .../join/RandomSortMergeOuterJoinTest.java         |  224 ++
 .../join/RowTimeBoundedStreamJoinTest.java         |  387 +++
 .../operators/join/SortMergeJoinIteratorTest.java  |  387 +++
 .../join/String2HashJoinOperatorTest.java          |  318 ++
 .../join/String2SortMergeJoinOperatorTest.java     |  221 ++
 .../join/TimeBoundedStreamJoinTestBase.java        |   64 +
 .../over/BufferDataOverWindowOperatorTest.java     |  237 ++
 .../over/NonBufferOverWindowOperatorTest.java      |  193 ++
 .../operators/over/SumAggsHandleFunction.java      |   89 +
 .../operators/rank/AppendOnlyTopNFunctionTest.java |   70 +
 .../rank/RetractableTopNFunctionTest.java          |  222 ++
 .../operators/rank/TopNFunctionTestBase.java       |  342 +++
 .../operators/rank/UpdatableTopNFunctionTest.java  |  226 ++
 .../operators/sort/BinaryExternalSorterTest.java   |  417 +++
 .../operators/sort/BinaryMergeIteratorTest.java    |  177 ++
 .../sort/BufferedKVExternalSorterTest.java         |  168 ++
 .../operators/sort/IntNormalizedKeyComputer.java   |   90 +
 .../operators/sort/IntRecordComparator.java        |   48 +
 .../operators/sort/ProcTimeSortOperatorTest.java   |  129 +
 .../operators/sort/RowTimeSortOperatorTest.java    |  209 ++
 .../table/runtime/operators/sort/SortUtilTest.java |  132 +
 .../operators/sort/StreamSortOperatorTest.java     |  111 +
 .../sort/StringNormalizedKeyComputer.java          |   72 +
 .../operators/sort/StringRecordComparator.java     |   40 +
 .../operators/sort/TestMemorySegmentPool.java      |   51 +
 .../operators/window/MergingWindowSetTest.java     |  596 ++++
 .../window/WindowOperatorContractTest.java         |  250 ++
 .../operators/window/WindowOperatorTest.java       | 1335 ++++++++
 .../runtime/operators/window/WindowTestUtils.java  |   33 +
 .../assigners/SessionWindowAssignerTest.java       |  148 +
 .../assigners/SlidingWindowAssignerTest.java       |  191 ++
 .../assigners/TumblingWindowAssignerTest.java      |   93 +
 .../window/grouping/HeapWindowsGroupingTest.java   |  394 +++
 .../operators/window/triggers/TriggersTest.java    |  119 +
 .../wmassigners/MiniBatchAssignerOperatorTest.java |  125 +
 .../MiniBatchedWatermarkAssignerOperatorTest.java  |  127 +
 .../wmassigners/WatermarkAssignerOperatorTest.java |  165 +
 .../WatermarkAssignerOperatorTestBase.java         |   64 +
 .../over/BufferDataOverWindowOperatorTest.java     |  237 --
 .../over/NonBufferOverWindowOperatorTest.java      |  193 --
 .../table/runtime/over/SumAggsHandleFunction.java  |   89 -
 .../runtime/rank/AppendOnlyTopNFunctionTest.java   |   70 -
 .../runtime/rank/RetractableTopNFunctionTest.java  |  222 --
 .../table/runtime/rank/TopNFunctionTestBase.java   |  342 ---
 .../runtime/rank/UpdatableTopNFunctionTest.java    |  226 --
 .../runtime/sort/BinaryExternalSorterTest.java     |  417 ---
 .../runtime/sort/BinaryMergeIteratorTest.java      |  177 --
 .../runtime/sort/BufferedKVExternalSorterTest.java |  168 --
 .../runtime/sort/IntNormalizedKeyComputer.java     |   90 -
 .../table/runtime/sort/IntRecordComparator.java    |   48 -
 .../runtime/sort/ProcTimeSortOperatorTest.java     |  129 -
 .../runtime/sort/RowTimeSortOperatorTest.java      |  209 --
 .../flink/table/runtime/sort/SortUtilTest.java     |  132 -
 .../table/runtime/sort/StreamSortOperatorTest.java |  111 -
 .../runtime/sort/StringNormalizedKeyComputer.java  |   72 -
 .../table/runtime/sort/StringRecordComparator.java |   40 -
 .../table/runtime/sort/TestMemorySegmentPool.java  |   51 -
 .../runtime/typeutils/BaseArraySerializerTest.java |  153 +
 .../runtime/typeutils/BaseMapSerializerTest.java   |  163 +
 .../runtime/typeutils/BaseRowSerializerTest.java   |  243 ++
 .../runtime/typeutils/BigDecimalTypeInfoTest.java  |   51 +
 .../typeutils/BinaryGenericSerializerTest.java     |   55 +
 .../runtime/typeutils/BinaryRowSerializerTest.java |   64 +
 .../runtime/typeutils/BinaryRowTypeInfoTest.java   |   79 +
 .../typeutils/BinaryStringSerializerTest.java      |   52 +
 .../runtime/typeutils/DecimalSerializerTest.java   |   53 +
 .../runtime/typeutils/InternalTypeInfoTest.java    |   36 +
 .../runtime/typeutils/SerializerTestUtil.java      |  143 +
 .../table/runtime/util/BaseRowHarnessAssertor.java |    2 +-
 .../table/runtime/util/BaseRowRecordEqualiser.java |    2 +-
 .../table/runtime/util/BinaryRowKeySelector.java   |    4 +-
 .../runtime/util/ResettableExternalBufferTest.java |    2 +-
 .../flink/table/runtime/util/SegmentsUtilTest.java |  204 ++
 .../MiniBatchAssignerOperatorTest.java             |  125 -
 .../MiniBatchedWatermarkAssignerOperatorTest.java  |  127 -
 .../WatermarkAssignerOperatorTest.java             |  165 -
 .../WatermarkAssignerOperatorTestBase.java         |   64 -
 .../table/runtime/window/MergingWindowSetTest.java |  597 ----
 .../runtime/window/WindowOperatorContractTest.java |  252 --
 .../table/runtime/window/WindowOperatorTest.java   | 1338 --------
 .../table/runtime/window/WindowTestUtils.java      |   35 -
 .../assigners/SessionWindowAssignerTest.java       |  148 -
 .../assigners/SlidingWindowAssignerTest.java       |  191 --
 .../assigners/TumblingWindowAssignerTest.java      |   93 -
 .../window/grouping/HeapWindowsGroupingTest.java   |  393 ---
 .../runtime/window/triggers/TriggersTest.java      |  119 -
 .../table/typeutils/BaseArraySerializerTest.java   |  153 -
 .../table/typeutils/BaseMapSerializerTest.java     |  163 -
 .../table/typeutils/BaseRowSerializerTest.java     |  243 --
 .../table/typeutils/BigDecimalTypeInfoTest.java    |   51 -
 .../typeutils/BinaryGenericSerializerTest.java     |   55 -
 .../table/typeutils/BinaryRowSerializerTest.java   |   64 -
 .../table/typeutils/BinaryRowTypeInfoTest.java     |   79 -
 .../typeutils/BinaryStringSerializerTest.java      |   52 -
 .../table/typeutils/DecimalSerializerTest.java     |   53 -
 .../table/typeutils/InternalTypeInfoTest.java      |   36 -
 .../flink/table/typeutils/SerializerTestUtil.java  |  143 -
 .../apache/flink/table/util/SegmentsUtilTest.java  |  203 --
 2628 files changed, 257054 insertions(+), 256826 deletions(-)
 create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
 create mode 100644 flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/OptimizerConfigOptions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/FlinkCalciteCatalogReader.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/FlinkReturnTypes.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericExceptFirstOperandChecker.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericOrDefaultReturnTypeInference.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/RepeatFamilyOperandTypeChecker.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FlinkSchema.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BlinkExecutorFactory.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/ExecutorBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/StreamExecutor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ExpressionBuilder.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggInputReference.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggLocalReference.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedDistinctKeyReference.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeExpression.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/SqlAggFunctionVisitor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/InternalFunctionDefinitions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CollectAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/Count1AggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CountAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DeclarativeAggregateFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DenseRankAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/RankAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/RankLikeAggFunctionBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/RowNumberAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/SingleValueAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/Sum0AggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/SumAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/SumWithRetractAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/CalciteSqlFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/FlinkSqlOperatorTable.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/ProctimeMaterializeSqlFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/SqlConcatAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/SqlFirstLastValueAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/SqlIncrSumAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/SqlThrowExceptionFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/StreamRecordTimestampSqlFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/sql/internal/SqlAuxiliaryGroupAggFunction.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/tablefunctions/ReplicateRows.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/DataStreamQueryOperation.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/PlannerQueryOperation.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/RichTableSourceQueryOperation.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/PartialFinalType.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/metadata/FlinkMetadata.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/metadata/FlinkRelMetadataQuery.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/exec/ExecNodeVisitor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/exec/ExecNodeVisitorImpl.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/process/DAGProcessContext.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/process/DAGProcessor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResource.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResourceUtil.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/FinalParallelismSetter.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ParallelismProcessor.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStage.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageGenerator.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculator.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkProjectJoinTransposeRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRule.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/rules/logical/SubQueryDecorrelator.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/JoinTypeUtil.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/KeySelectorUtil.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/OperatorType.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/BlinkPlannerFactory.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/PlannerContext.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FlinkSchema.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/FunctionCatalogOperatorTable.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BatchExecutor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BlinkExecutorFactory.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/BlinkPlannerFactory.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ExecutorBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/StreamExecutor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ExpressionBuilder.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerTypeInferenceUtilImpl.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggInputReference.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggLocalReference.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedDistinctKeyReference.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeConverter.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeExpression.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/SqlAggFunctionVisitor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/InternalFunctionDefinitions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/AvgAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CollectAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/ConcatAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/ConcatWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/Count1AggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/CountAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DeclarativeAggregateFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/DenseRankAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/IncrSumAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/IncrSumWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LeadLagAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/MaxAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/MinAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/MinWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/RankAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/RankLikeAggFunctionBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/RowNumberAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/SingleValueAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/Sum0AggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/SumAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/SumWithRetractAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/CalciteSqlFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/ProctimeMaterializeSqlFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlConcatAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlFirstLastValueAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlIncrSumAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlThrowExceptionFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/StreamRecordTimestampSqlFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/internal/SqlAuxiliaryGroupAggFunction.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/tablefunctions/ReplicateRows.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/DataStreamQueryOperation.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/PlannerQueryOperation.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/RichTableSourceQueryOperation.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlConversionException.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/PartialFinalType.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkMetadata.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/metadata/FlinkRelMetadataQuery.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeVisitor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeVisitorImpl.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/process/DAGProcessContext.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/process/DAGProcessor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResource.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/NodeResourceUtil.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/FinalParallelismSetter.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ParallelismProcessor.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStage.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageGenerator.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculator.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkProjectJoinTransposeRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRule.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/type/FlinkReturnTypes.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/type/NumericExceptFirstOperandChecker.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/type/NumericOrDefaultReturnTypeInference.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/type/RepeatFamilyOperandTypeChecker.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/JoinTypeUtil.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/KeySelectorUtil.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/utils/OperatorType.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/AggregatePhaseStrategy.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/SingleElementIterator.java
 create mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlConversionException.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/SingleElementIterator.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkContext.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkContextImpl.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLocalRef.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLogicalRelFactories.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelFactories.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptCluster.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelOptClusterFactory.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/PreValidateReWriter.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CalcCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenException.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGeneratorContext.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CorrelateCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/EqualiserCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpandCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExprCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GenerateUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GeneratedExpression.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/HashCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/Indenter.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LongHashJoinGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LookupJoinCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/OperatorCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ProjectionCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/SinkCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ValuesCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggCodeGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DeclarativeAggCodeGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DistinctAggCodeGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/ImperativeAggCodeGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggCodeGenHelper.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenHelper.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashWindowCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortWindowCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/WindowCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/CallGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ConstantCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/CurrentTimePointCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/DivCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ExtractCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FloorCeilCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/HashCodeCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/IfCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/LikeCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/MethodCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/PrintCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/RandCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TimestampDiffCallGen.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/RangeBoundComparatorCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/ComparatorCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/SortCodeGenerator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewSpec.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ExpressionBridge.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/InputTypeSpec.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpression.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/Reinterpret.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ReturnTypeInference.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/call.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/cast.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/collection.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/comparison.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/composite.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/hashExpressions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/literals.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/logic.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ordering.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/overOffsets.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/plannerWindowProperties.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/stringExpressions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/subquery.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/symbols.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/time.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/windowProperties.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/package.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/TreeNode.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/cost/FlinkCost.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/cost/FlinkCostBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/cost/FlinkCostFactory.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/cost/FlinkCostFactoryBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/logical/MatchRecognize.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/AggCallSelectivityEstimator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkDefaultRelMetadataProvider.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnInterval.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCount.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnOriginNullCount.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniqueness.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdCumulativeCost.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCount.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistribution.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdFilteredColumnInterval.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicity.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdNonCumulativeCost.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdPercentageOriginalRows.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdPopulationSize.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCount.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdSelectivity.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdSize.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueGroups.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeys.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/SelectivityEstimator.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/FlinkConventions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRelNode.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Expand.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalExpand.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalRank.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalWatermarkAssigner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Rank.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Sink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/WatermarkAssigner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/WindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonCalc.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonIntermediateTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalExchange.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/exec/BatchExecNode.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/exec/ExecNode.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/exec/StreamExecNode.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCorrelate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalExpand.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalIntermediateTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalIntersect.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMinus.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalOverAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRank.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRel.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSnapshot.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalUnion.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalValues.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWatermarkAssigner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/FlinkPhysicalRel.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/PhysicalTableSourceScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecBoundedStreamScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCorrelate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExpand.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecIntermediateTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLimit.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLookupJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortLimit.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecTableSourceScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecUnion.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecValues.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecWindowAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchPhysicalRel.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCalc.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCorrelate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDataStreamScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExchange.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExpand.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregateBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIntermediateTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLookupJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecMatch.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecOverAggregate.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalSort.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecUnion.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecValues.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWindowJoin.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamPhysicalRel.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/CommonSubGraphBasedOptimizer.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/Optimizer.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/BatchOptimizeContext.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkBatchProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkChainedProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkDecorrelateProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkGroupProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkHepProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkHepRuleSetProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkMiniBatchIntervalTraitInitProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkOptimizeContext.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkOptimizeProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkRelTimeIndicatorProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkRuleSetProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkStreamProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkUpdateAsRetractionTraitInitProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkVolcanoProgram.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/StreamOptimizeContext.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/DeadlockBreakupProcessor.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/BatchLogicalWindowAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcSnapshotTransposeRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/DecomposeGroupingSetsRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/EnumerableToLogicalTableScan.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRules.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkRewriteSubQueryRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkSubQueryRemoveRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalWindowAggregateRuleBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/StreamLogicalWindowAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowPropertiesRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/FlinkExpandConversionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecBoundedStreamScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecCalcRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecConstantTableFunctionScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecCorrelateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecExpandRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecIntermediateTableScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLimitRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLookupJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRuleBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecOverAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecRankRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSingleRowJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSinkRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortLimitRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecTableSourceScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecUnionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecValuesRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/common/CommonLookupJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/IncrementalAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecCalcRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecConstantTableFunctionScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecCorrelateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDataStreamScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDeduplicateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecExpandRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecGroupAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecGroupWindowAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecIntermediateTableScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLimitRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLookupJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecMatchRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecOverAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecRankRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecRetractionRules.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecSinkRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecSortLimitRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecSortRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecTemporalJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecTemporalSortRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecUnionRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecValuesRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecWatermarkAssignerRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecWindowJoinRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DeferredTypeFlinkTableFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/FlinkRelOptTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/FlinkTableFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/GenericRelDataType.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/InlineTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/IntermediateRelTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataType.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TypedFlinkTableFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/ValueInterval.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/FlinkRelDistribution.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/FlinkRelDistributionTraitDef.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/MiniBatchIntervalTrait.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/MiniBatchIntervalTraitDef.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/RelModifiedMonotonicity.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/TraitUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/retractionTraitDefs.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/trait/retractionTraits.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggFunctionFactory.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ColumnIntervalUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/CorrelateUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ExecNodePlanDumper.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ExpandUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ExplodeFunctionUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelMdUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelOptUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRexUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/JoinUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/LookupJoinUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/MatchUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/OverAggregateUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/PartitionPruner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankProcessStrategy.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RelDigestUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RelExplainUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RelShuttles.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RelTreeWriterImpl.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexDefaultVisitor.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeExtractor.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeRewriter.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ScanUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SetOpRewriteUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SortUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/TemporalJoinUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/aggregation.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/BatchPlanner.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/PlannerBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/CalciteConfig.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkCalciteSqlValidator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContext.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkContextImpl.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLocalRef.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelFactories.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptCluster.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelOptClusterFactory.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeSystem.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CalcCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenException.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CollectorCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/EqualiserCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpandCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/FunctionCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GeneratedExpression.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/HashCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/Indenter.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/InputFormatCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/LongHashJoinGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/NestedLoopJoinCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/OperatorCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/SinkCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ValuesCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggCodeGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/DeclarativeAggCodeGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/DistinctAggCodeGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/ImperativeAggCodeGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggCodeGenHelper.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenHelper.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/HashWindowCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/SortWindowCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/agg/batch/WindowCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/BuiltInMethods.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/CallGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ConstantCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/CurrentTimePointCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/DivCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ExtractCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FloorCeilCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/FunctionGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/HashCodeCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/IfCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/MethodCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/PrintCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/RandCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarFunctionCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/StringCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TableFunctionCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/calls/TimestampDiffCallGen.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/over/RangeBoundComparatorCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/ComparatorCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/sort/SortCodeGenerator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/dataview/DataViewSpec.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/dataview/DataViewUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/ExpressionBridge.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/InputTypeSpec.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpression.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/Reinterpret.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/ReturnTypeInference.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/aggregations.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/arithmetic.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/call.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/cast.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/collection.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/comparison.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/composite.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/hashExpressions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/literals.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/logic.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/mathExpressions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/ordering.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/overOffsets.scala
 rename flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/{ => planner}/expressions/package.scala (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/plannerWindowProperties.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/stringExpressions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/subquery.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/symbols.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/time.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/AggSqlFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/ScalarSqlFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/TableSqlFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/functions/utils/UserDefinedFunctionUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/package.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/TreeNode.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/cost/FlinkCost.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/cost/FlinkCostBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/cost/FlinkCostFactory.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/cost/FlinkCostFactoryBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/logical/MatchRecognize.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/logical/groupWindows.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkDefaultRelMetadataProvider.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnInterval.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCount.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdCumulativeCost.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnInterval.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicity.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdNonCumulativeCost.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPercentageOriginalRows.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSize.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivity.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSize.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroups.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/FlinkConventions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/FlinkRelNode.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Expand.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalExpand.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalRank.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWatermarkAssigner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Rank.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WatermarkAssigner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonCalc.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonIntermediateTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonLookupJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalExchange.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/common/CommonPhysicalJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/exec/BatchExecNode.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/exec/StreamExecNode.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalCalc.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalCorrelate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalDataStreamTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalExpand.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalIntermediateTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalIntersect.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMatch.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalMinus.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalOverAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalRank.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalRel.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSnapshot.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalUnion.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalValues.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWatermarkAssigner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/FlinkPhysicalRel.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/PhysicalTableSourceScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecBoundedStreamScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCalc.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecCorrelate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExchange.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecExpand.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecIntermediateTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecJoinBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLimit.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLocalHashWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLocalSortWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecLookupJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecOverAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecRank.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSort.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortLimit.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecTableSourceScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecUnion.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecValues.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecWindowAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRel.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCalc.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecCorrelate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDataStreamScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecDeduplicate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecExchange.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecExpand.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupAggregateBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecIntermediateTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLimit.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecLookupJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecMatch.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecOverAggregate.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecRank.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSort.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecSortLimit.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTemporalJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecTemporalSort.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecUnion.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecValues.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamExecWindowJoin.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRel.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/CommonSubGraphBasedOptimizer.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/Optimizer.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/RelNodeBlock.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/BatchOptimizeContext.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkDecorrelateProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkGroupProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkHepProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkHepRuleSetProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkMiniBatchIntervalTraitInitProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkOptimizeContext.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkOptimizeProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkRelTimeIndicatorProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkRuleSetProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkUpdateAsRetractionTraitInitProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkVolcanoProgram.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/StreamOptimizeContext.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/DeadlockBreakupProcessor.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/BatchLogicalWindowAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CalcSnapshotTransposeRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/EnumerableToLogicalTableScan.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRules.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkRewriteSubQueryRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSubQueryRemoveRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableFunctionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalWindowAggregateRuleBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteCoalesceRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/StreamLogicalWindowAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowAggregateReduceFunctionsRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/WindowPropertiesRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/FlinkExpandConversionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecAggRuleBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecBoundedStreamScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCalcRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecConstantTableFunctionScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecCorrelateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecExpandRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashAggRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecHashJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecIntermediateTableScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecLimitRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecLookupJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecNestedLoopJoinRuleBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecOverAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecRankRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSingleRowJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSinkRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortLimitRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecTableSourceScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecUnionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecValuesRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/common/CommonLookupJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCalcRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecConstantTableFunctionScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecCorrelateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecDataStreamScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecDeduplicateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecExpandRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecGroupAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecGroupWindowAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecIntermediateTableScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecLimitRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecLookupJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecMatchRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecOverAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecRankRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecRetractionRules.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSinkRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSortLimitRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecSortRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTableSourceScanRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTemporalJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecTemporalSortRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecUnionRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecValuesRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecWatermarkAssignerRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamExecWindowJoinRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/DataStreamTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/DeferredTypeFlinkTableFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkRelOptTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/FlinkTableFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/GenericRelDataType.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/InlineTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/IntermediateRelTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSinkTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceSinkTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TableSourceTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TypedFlinkTableFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/ValueInterval.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/FlinkRelDistribution.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/FlinkRelDistributionTraitDef.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/MiniBatchIntervalTrait.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/MiniBatchIntervalTraitDef.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/RelModifiedMonotonicity.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/TraitUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/retractionTraitDefs.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/trait/retractionTraits.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/ColumnIntervalUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/CorrelateUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/ExecNodePlanDumper.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/ExpandUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/ExplodeFunctionUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/JoinUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/LookupJoinUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/MatchUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/OverAggregateUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankProcessStrategy.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelDigestUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelExplainUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelShuttles.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexDefaultVisitor.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriter.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/ScanUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/SortUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/TemporalJoinUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/UpdatingPlanChecker.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowEmitStrategy.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/aggregation.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/CollectTableSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/DataStreamTableSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sinks/TableSinkUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/sources/TableSourceUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/typeutils/TypeCoercion.scala
 rename flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/{ => planner}/typeutils/TypeInfoCheckUtils.scala (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/JavaScalaConversionUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/Logging.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/utils/PlanUtil.scala
 rename flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/{ => planner}/validate/ValidationResult.scala (100%)
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/CollectTableSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/DataStreamTableSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/JavaScalaConversionUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/Logging.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/PlanUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/calcite/FlinkCalciteCatalogReaderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/LongHashJoinGeneratorTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/SortCodeGeneratorTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/agg/TestLongAvgFunc.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/AggFunctionTestBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunctionTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunctionTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunctionTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunctionTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/MockNodeTestBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/FinalParallelismSetterTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageGeneratorTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/util/JavaUserDefinedAggFunctions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/LongHashJoinGeneratorTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/SortCodeGeneratorTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/codegen/agg/TestLongAvgFunc.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/AggFunctionTestBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ConcatWithRetractAggFunctionTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/ConcatWsWithRetractAggFunctionTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstLastValueAggFunctionWithOrderTestBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MaxWithRetractAggFunctionTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/functions/aggfunctions/MinWithRetractAggFunctionTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReaderTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/MockNodeTestBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/FinalParallelismSetterTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageGeneratorTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/utils/JavaUserDefinedAggFunctions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/BatchAbstractTestBase.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/FailingCollectionSource.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaPojos.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedAggFunctions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedScalarFunctions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/utils/RangeInputFormat.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/BaseRowTestUtil.java
 create mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/utils/DiffRepository.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/BatchAbstractTestBase.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/FailingCollectionSource.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/JavaPojos.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedAggFunctions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedScalarFunctions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedTableFunctions.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/runtime/utils/RangeInputFormat.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/BaseRowTestUtil.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/DiffRepository.java
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/table/CalcTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/table/CorrelateTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/table/JoinTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/table/SetOperatorsTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/SetOperatorsTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/table/CalcTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/table/ColumnFunctionsTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/table/CorrelateTest.xml
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/table/OverWindowTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{plan/stream => planner/plan/batch}/sql/CalcTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/DagOptimizationTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/DeadlockBreakupTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/LimitTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/RankTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/RemoveCollationTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/RemoveShuffleTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/SinkTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/SortLimitTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/SortTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/SubplanReuseTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/TableScanTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/TableSourceTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{plan/stream => planner/plan/batch}/sql/UnionTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/UnnestTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/ValuesTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/AggregateReduceGroupingTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/DistinctAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/GroupingSetsTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/HashAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/OverAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/SortAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/agg/WindowAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/BroadcastHashJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/JoinReorderTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/LookupJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/NestedLoopJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/SemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/ShuffledHashJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/SingleRowJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/SortMergeJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/table/AggregateTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CalcTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/table/ColumnFunctionsTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/table/GroupWindowTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/JoinTest.xml
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/batch/table/stringexpr/SetOperatorsTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/nodes/resource/ExecNodeResourceTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/AggregateReduceGroupingRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/CalcRankTransposeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ConvertToNotInOrInRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/DecomposeGroupingSetsRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ExpressionReductionRulesTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkAggregateRemoveRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkCalcMergeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkFilterJoinRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkLimit0RemoveRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkPruneEmptyRulesTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/JoinConditionEqualityTransferRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/JoinDependentConditionDerivationRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/JoinDeriveNullFilterRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/LogicalUnnestRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/RankNumberColumnRemoveRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/RewriteCoalesceRuleTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/RewriteMultiJoinConditionRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/SimplifyFilterConditionRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/SimplifyJoinConditionRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/SplitAggregateRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/WindowGroupReorderRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/physical/stream/RetractionRulesTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{plan/batch => planner/plan/stream}/sql/CalcTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/DagOptimizationTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/LimitTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/MiniBatchIntervalInferTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/ModifiedMonotonicityTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/RankTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/RelTimeIndicatorConverterTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/SinkTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/SortLimitTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/SortTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/SubplanReuseTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/TableScanTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/TableSourceTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{plan/batch => planner/plan/stream}/sql/UnionTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/UnnestTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/ValuesTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/AggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/DistinctAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/GroupingSetsTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/IncrementalAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/OverAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/TwoStageAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/agg/WindowAggregateTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/JoinReorderTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/JoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/LookupJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/SemiAntiJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/TemporalJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/sql/join/WindowJoinTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/AggregateTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CalcTest.xml
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/GroupWindowTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/JoinTest.xml (100%)
 create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/OverWindowTest.xml
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/SetOperatorsTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/TableSourceTest.xml (100%)
 rename flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/{ => planner}/plan/stream/table/TwoStageAggregateTest.xml (100%)
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/FlinkTypeFactoryTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/HashCodeGeneratorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/ProjectionCodeGeneratorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGeneratorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/BatchAggTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGeneratorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGeneratorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/ArrayTypeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/CompositeAccessTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/KeywordParseTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/LiteralTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/MapTypeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/MathFunctionsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/RowTypeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/SqlExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ArrayTypeTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/CompositeTypeTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/MapTypeTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarTypesTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/userDefinedScalarFunctions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/validation/ArrayTypeValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/validation/CompositeAccessValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/validation/MapTypeValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/validation/RowTypeValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/factories/utils/TestCollectionTableFactory.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/match/PatternTranslatorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/CalcTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/LimitTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RankTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableScanTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableSourceTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/UnionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/UnnestTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/ValuesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateReduceGroupingTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/DistinctAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/GroupingSetsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/JoinReorderTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/JoinTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SingleRowJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/TemporalJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/AggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/CalcTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/ColumnFunctionsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/CorrelateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/GroupWindowTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/JoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/SetOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/TemporalTableJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/AggregateStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/CalcStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/JoinStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/SetOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/stringexpr/SortStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/AggregateValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/CalcValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/CorrelateValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/GroupWindowValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/JoinValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/OverWindowValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/SetOperatorsValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/table/validation/SortValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/UnnestTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/cost/FlinkCostTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/AggCallSelectivityEstimatorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnIntervalTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCountTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniquenessTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdCumulativeCostTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCountTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistributionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdHandlerTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdNonCumulativeCostTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdPercentageOriginalRowsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdPopulationSizeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCountTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdSelectivityTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdSizeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueGroupsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeysTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataHandlerConsistencyTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataTestUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/SelectivityEstimatorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/optimize/program/FlinkChainedProgramTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/optimize/program/FlinkHepRuleSetProgramTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/optimize/program/FlinkVolcanoProgramTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ExpressionReductionRulesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRulesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PruneAggregateCallRuleTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/schema/TimeIndicatorRelDataTypeTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stats/ValueIntervalTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/CalcTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/LimitTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/RankTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/RelTimeIndicatorConverterTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SetOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SinkTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SortLimitTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SortTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/TableScanTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/TableSourceTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/UnionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/UnnestTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ValuesTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/GroupingSetsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/IncrementalAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/OverAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/WindowAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/JoinReorderTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/JoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/LookupJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/SemiAntiJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/TemporalJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/WindowJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/AggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/CalcTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/ColumnFunctionsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/CorrelateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/GroupWindowTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/JoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/OverWindowTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/SetOperatorsTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TableSourceTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TemporalTableJoinTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TwoStageAggregateTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/AggregateStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/CalcStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/GroupWindowTableAggregateStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/OverWindowStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/stringexpr/SetOperatorsStringExpressionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/AggregateValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/CalcValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/CorrelateValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/GroupWindowValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/OverWindowValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/SetOperatorsValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/TableSinkValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/TemporalTableJoinValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/validation/UnsupportedOpsValidationTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/trait/FlinkRelDistributionTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/FlinkRelOptUtilTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/FlinkRexUtilTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/InputTypeBuilder.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/PartitionPrunerTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RelDigestUtilTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeExtractorTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeRewriterTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/pojos.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactoryTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/HashCodeGeneratorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGeneratorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/AggTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/AggsHandlerCodeGeneratorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/AggWithoutKeysTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/BatchAggTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/HashAggCodeGeneratorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/agg/batch/SortAggCodeGeneratorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/KeywordParseTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/SqlExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ArrayTypeTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/CompositeTypeTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/MapTypeTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/RowTypeTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarTypesTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/match/PatternTranslatorTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LimitTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RankTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableScanTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/UnnestTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/ValuesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateReduceGroupingTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/AggregateTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/DistinctAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupingSetsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/OverAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinReorderTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/TemporalJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CalcTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/ColumnFunctionsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/GroupWindowTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/JoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/TemporalTableJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/AggregateStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CalcStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/JoinStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/SetOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/SortStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/AggregateValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CalcValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/GroupWindowValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/JoinValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/OverWindowValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SetOperatorsValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/SortValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/AggregateReduceGroupingTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/UnnestTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/cost/FlinkCostTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/AggCallSelectivityEstimatorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnIntervalTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCountTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnOriginNullCountTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniquenessTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdCumulativeCostTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistributionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdFilteredColumnIntervalTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdNonCumulativeCostTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPercentageOriginalRowsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSizeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCountTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSizeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueGroupsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeysTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataHandlerConsistencyTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimatorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/nodes/resource/ExecNodeResourceTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChainedProgramTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkHepRuleSetProgramTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkVolcanoProgramTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/CalcRankTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/DecomposeGroupingSetsRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ExpressionReductionRulesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateJoinTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkCalcMergeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForConstantRangeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkLogicalRankRuleForRangeEndTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkPruneEmptyRulesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PruneAggregateCallRuleTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushPartitionIntoTableSourceScanRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RankNumberColumnRemoveRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteCoalesceRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/WindowGroupReorderRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/FlinkRewriteSubQueryRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubqueryCorrelateVariablesValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataTypeTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stats/ValueIntervalTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/CalcTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/LimitTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SinkTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnnestTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ValuesTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupingSetsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/OverAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinReorderTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/AggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CalcTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/GroupWindowTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/JoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/OverWindowTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/SetOperatorsTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TemporalTableJoinTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TwoStageAggregateTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/AggregateStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CalcStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/CorrelateStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/GroupWindowStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/GroupWindowTableAggregateStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/OverWindowStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/stringexpr/SetOperatorsStringExpressionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/AggregateValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CalcValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/GroupWindowValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/OverWindowValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/SetOperatorsValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/UnsupportedOpsValidationTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/trait/FlinkRelDistributionTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtilTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtilTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/InputTypeBuilder.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/PartitionPrunerTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RelDigestUtilTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriterTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/pojos.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CorrelateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CorrelateITCase2.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/DecimalITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/Limit0RemoveITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/MiscITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/OverWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/RankITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SetOperatorsITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/SortLimitITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableScanITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnionITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/UnnestITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/ValuesITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateITCaseBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateRemoveITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/GroupingSetsITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashAggITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/PruneAggregateCallITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortAggITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/WindowAggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/InnerJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinConditionTypeCoerceITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCaseHelper.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinWithoutKeyITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/LookupJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/OuterJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/ScalarQueryITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/SemiJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/AggregationITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CalcITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/DecimalITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/GroupWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/JoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/OverWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/SetOperatorsITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/SortITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverWindowHarnessTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateRemoveITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AsyncLookupJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/DeduplicateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/LimitITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/LookupJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/OverWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/PruneAggregateCallITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SemiAntiJoinStreamITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SetOperatorsITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SortLimitITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SplitAggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableScanITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalSortITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/ValuesITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowJoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/CalcITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/CorrelateITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/GroupWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/MiniBatchGroupWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/OverWindowITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/RetractionITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/SetOperatorsITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/SubQueryITCase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/CollectionBatchExecTable.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/InMemoryLookupableTableSource.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/SortTestUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTableEnvUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamTestSink.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithAggTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithStateTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestSinkUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TimeTestUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/UserDefinedFunctionTestUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/AvgAggFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/ColumnIntervalUtilTest.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/CountAggFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/LogicalPlanFormatUtils.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/MemoryTableSourceSinkUtil.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/SumAggFunction.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedAggFunctions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala
 create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/testTableSources.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase2.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/Limit0RemoveITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/LimitITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SetOperatorsITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableScanITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableSourceITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnnestITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/ValuesITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateITCaseBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateRemoveITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/DistinctAggregateITCaseBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/GroupingSetsITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/PruneAggregateCallITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/WindowAggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinConditionTypeCoerceITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinWithoutKeyITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/LookupJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/OuterJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/ScalarQueryITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/SemiJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregationITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/GroupWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/JoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/OverWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/SetOperatorsITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/SortITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateRemoveITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AsyncLookupJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CorrelateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/DeduplicateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/JoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/Limit0RemoveITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/LimitITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/LookupJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/MatchRecognizeITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/PruneAggregateCallITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SemiAntiJoinStreamITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SetOperatorsITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortLimitITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TableScanITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TableSourceITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalSortITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/UnnestITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/ValuesITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowAggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowJoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/CalcITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/CorrelateITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/GroupWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/MiniBatchGroupWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/OverWindowITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/RetractionITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/SetOperatorsITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/SubQueryITCase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/CollectionBatchExecTable.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/InMemoryLookupableTableSource.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/SortTestUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTableEnvUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestSink.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TableUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestData.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TimeTestUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/UserDefinedFunctionTestUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/AvgAggFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/ColumnIntervalUtilTest.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/CountAggFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/DateTimeTestUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/LogicalPlanFormatUtils.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/MemoryTableSourceSinkUtil.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/SumAggFunction.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/UserDefinedAggFunctions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/UserDefinedTableFunctions.scala
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/testTableSources.scala
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/ExecutionConfigOptions.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/CountWindow.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/TimeWindow.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/Window.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/NullAwareMapIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerKeyStateDataViewStore.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerWindowStateDataViewStore.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataViewStore.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateListView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateMapView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/AggsHandleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/CompileUtils.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedAggsHandleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedClass.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedCollector.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedHashFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedInput.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedJoinCondition.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNamespaceAggsHandleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNormalizedKeyComputer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedProjection.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordComparator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordEqualiser.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedResultFuture.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/HashFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/JoinCondition.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NamespaceAggsHandleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NormalizedKeyComputer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/Projection.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordComparator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordEqualiser.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/AbstractProcessStreamOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/CodeGenOperatorFactory.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/TableStreamOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMap.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMapSpillMemorySegmentPool.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/GroupAggFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGlobalGroupAggFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGroupAggFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchIncrementalGroupAggFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchLocalGroupAggFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/RecordCounter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/AbstractMapBundleOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/KeyedMapBundleOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTrigger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTriggerCallback.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CoBundleTrigger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTrigger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTrigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/NullAwareMapIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerKeyStateDataViewStore.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerWindowStateDataViewStore.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataView.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataViewStore.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateListView.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionHelper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/AggsHandleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/CompileUtils.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedAggsHandleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedClass.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedCollector.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedHashFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedInput.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedJoinCondition.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNamespaceAggsHandleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNormalizedKeyComputer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedProjection.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordComparator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordEqualiser.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedResultFuture.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/HashFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/JoinCondition.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NamespaceAggsHandleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NormalizedKeyComputer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/Projection.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordComparator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordEqualiser.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/EmitAwareCollector.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/FlinkJoinType.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinType.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/KeyedCoProcessOperatorWithWatermarkDelay.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/NullAwareJoinHelper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/OuterJoinPaddingUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamJoin.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/RowTimeBoundedStreamJoin.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeFullOuterJoinIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeInnerJoinIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeOneSideOuterJoinIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinWithCalcRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/DelegatingResultFuture.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinWithCalcRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/AbstractStreamingJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingSemiAntiJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinInputSideSpec.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateViews.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateView.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateViews.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalProcessTimeJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalRowTimeJoinOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/BaseRowEventComparator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/IterativeConditionRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/PatternProcessFunctionRunner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/RowtimeProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/AbstractProcessStreamOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapSpillMemorySegmentPool.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGlobalGroupAggFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGroupAggFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchIncrementalGroupAggFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchLocalGroupAggFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/RecordCounter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/AbstractMapBundleOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/KeyedMapBundleOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTrigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTriggerCallback.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CoBundleTrigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTrigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTrigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionHelper.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/EmitAwareCollector.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/FlinkJoinType.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinType.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/NullAwareJoinHelper.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/OuterJoinPaddingUtil.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/ProcTimeBoundedStreamJoin.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/RowTimeBoundedStreamJoin.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeFullOuterJoinIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeInnerJoinIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeOneSideOuterJoinIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/TimeBoundedStreamJoin.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingSemiAntiJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinInputSideSpec.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateView.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateViews.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateView.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateViews.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/BaseRowEventComparator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/NonBufferOverWindowOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeUnboundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsUnboundedPrecedingFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/InsensitiveOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OffsetOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OverWindowFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeSlidingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedFollowingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedPrecedingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowSlidingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedFollowingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedPrecedingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/SlidingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedFollowingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedOverWindowFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedPrecedingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRange.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRangeWithoutEnd.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankRange.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankType.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/TopNBuffer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/VariableRankRange.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/AbstractBinaryExternalMerger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BaseTemporalSortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalMerger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryInMemorySortBuffer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryIndexedSortable.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVExternalMerger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVInMemorySortBuffer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryMergeIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ChannelReaderKVInputViewIterator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/LimitOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ListMemorySegmentPool.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ProcTimeSortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RankOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RowTimeSortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortLimitOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SpillChannelManager.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/StreamSortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/values/ValuesInputFormat.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/CountWindow.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/Window.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountSlidingWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountTumblingWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/InternalTimeWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/MergingWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/PanedWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SessionWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SlidingWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/TumblingWindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/WindowAssigner.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/HeapWindowsGrouping.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/WindowsGrouping.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/GeneralWindowProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowSet.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/PanedWindowProcessFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ElementTriggers.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchAssignerOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchedWatermarkAssignerOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/AbstractRowTimeUnboundedPrecedingOver.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/BufferDataOverWindowOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/NonBufferOverWindowOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRangeBoundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRowsBoundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeUnboundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeBoundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeUnboundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsBoundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsUnboundedPrecedingFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/InsensitiveOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OffsetOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OverWindowFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeSlidingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedFollowingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedPrecedingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowSlidingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedFollowingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedPrecedingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/SlidingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedFollowingOverFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedOverWindowFrame.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedPrecedingOverFrame.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/partitioner/BinaryHashPartitioner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AbstractTopNFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AppendOnlyTopNFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRange.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRangeWithoutEnd.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankRange.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankType.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RetractableTopNFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/TopNBuffer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/UpdatableTopNFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/VariableRankRange.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/AbstractBinaryExternalMerger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BaseTemporalSortOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalMerger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryInMemorySortBuffer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryIndexedSortable.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVExternalMerger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVInMemorySortBuffer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryMergeIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ChannelReaderKVInputViewIterator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/LimitOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ListMemorySegmentPool.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ProcTimeSortOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RankOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RowTimeSortOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortLimitOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SpillChannelManager.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/StreamSortOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassDataTypeConverter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoLogicalTypeConverter.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractRowSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BigDecimalTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryRowSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfoFactory.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfoFactory.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializerSnapshot.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapTypeInfo.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/MurmurHashUtil.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/SegmentsUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/values/ValuesInputFormat.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchAssignerOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchedWatermarkAssignerOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperatorBuilder.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountSlidingWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountTumblingWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/InternalTimeWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/MergingWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/PanedWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SessionWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SlidingWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/TumblingWindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/WindowAssigner.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/HeapWindowsGrouping.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/WindowsGrouping.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/GeneralWindowProcessFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/InternalWindowProcessFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowProcessFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowSet.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/PanedWindowProcessFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ElementTriggers.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/EventTimeTriggers.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ProcessingTimeTriggers.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/Trigger.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassDataTypeConverter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassLogicalTypeConverter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/InternalSerializers.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/LogicalTypeDataTypeConverter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/PlannerTypeUtils.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoDataTypeConverter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoLogicalTypeConverter.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractRowSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseArraySerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseMapSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BigDecimalTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryGenericSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryRowSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfoFactory.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfoFactory.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializerSnapshot.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapTypeInfo.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/TypeCheckUtils.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/MurmurHashUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/SegmentsUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/CompileUtilsTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedCollectorWrapper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedFunctionWrapper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedResultFutureWrapper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/BytesHashMapTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/HashAggTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/SumHashAggTestOperator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/MapBundleOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTriggerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTriggerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/TestTriggerCallback.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionTestBase.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/CompileUtilsTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedCollectorWrapper.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedFunctionWrapper.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedResultFutureWrapper.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/AsyncLookupJoinHarnessTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2HashJoinOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2SortMergeJoinOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/LookupJoinHarnessTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamJoinTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/RandomSortMergeInnerJoinTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/RandomSortMergeOuterJoinTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/RowTimeBoundedStreamJoinTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/SortMergeJoinIteratorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/String2HashJoinOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/String2SortMergeJoinOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/TimeBoundedStreamJoinTestBase.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/SumHashAggTestOperator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTriggerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTriggerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/TestTriggerCallback.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionTestBase.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2HashJoinOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/LookupJoinHarnessTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/ProcTimeBoundedStreamJoinTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/RandomSortMergeInnerJoinTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/RandomSortMergeOuterJoinTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/RowTimeBoundedStreamJoinTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIteratorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/String2HashJoinOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/String2SortMergeJoinOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/TimeBoundedStreamJoinTestBase.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/NonBufferOverWindowOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/SumAggsHandleFunction.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/TopNFunctionTestBase.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunctionTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorterTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BinaryMergeIteratorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorterTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/IntNormalizedKeyComputer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/IntRecordComparator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/ProcTimeSortOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/RowTimeSortOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/SortUtilTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/StreamSortOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/StringNormalizedKeyComputer.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/StringRecordComparator.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/sort/TestMemorySegmentPool.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/MergingWindowSetTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorContractTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowTestUtils.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/assigners/SessionWindowAssignerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/assigners/SlidingWindowAssignerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/assigners/TumblingWindowAssignerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/grouping/HeapWindowsGroupingTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/triggers/TriggersTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchAssignerOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchedWatermarkAssignerOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTestBase.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/over/BufferDataOverWindowOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/over/NonBufferOverWindowOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/over/SumAggsHandleFunction.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/rank/AppendOnlyTopNFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/rank/RetractableTopNFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/rank/TopNFunctionTestBase.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/rank/UpdatableTopNFunctionTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryExternalSorterTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryMergeIteratorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorterTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/IntNormalizedKeyComputer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/IntRecordComparator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/ProcTimeSortOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/RowTimeSortOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/SortUtilTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/StreamSortOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/StringNormalizedKeyComputer.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/StringRecordComparator.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/TestMemorySegmentPool.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BaseRowSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BigDecimalTypeInfoTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryRowSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryRowTypeInfoTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/BinaryStringSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/DecimalSerializerTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/InternalTypeInfoTest.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/typeutils/SerializerTestUtil.java
 create mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/SegmentsUtilTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchAssignerOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchedWatermarkAssignerOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperatorTestBase.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/MergingWindowSetTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/WindowOperatorContractTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/WindowOperatorTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/WindowTestUtils.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/assigners/SessionWindowAssignerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/assigners/SlidingWindowAssignerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/assigners/TumblingWindowAssignerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/grouping/HeapWindowsGroupingTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/window/triggers/TriggersTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BaseArraySerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BaseMapSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BaseRowSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BigDecimalTypeInfoTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BinaryGenericSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BinaryRowSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BinaryRowTypeInfoTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/BinaryStringSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/DecimalSerializerTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/InternalTypeInfoTest.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/typeutils/SerializerTestUtil.java
 delete mode 100644 flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/util/SegmentsUtilTest.java


[flink] 02/03: [FLINK-13266][table] Relocate blink runtime classes to avoid class clashes

Posted by ja...@apache.org.
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 9a6ca547d6bd261730c46519f6bffa0b699ec218
Author: godfreyhe <go...@163.com>
AuthorDate: Fri Jul 19 20:58:37 2019 +0800

    [FLINK-13266][table] Relocate blink runtime classes to avoid class clashes
---
 .../table/api/config/ExecutionConfigOptions.java   |  2 +-
 .../catalog/QueryOperationCatalogViewTable.java    |  2 +-
 .../expressions/PlannerTypeInferenceUtilImpl.java  |  2 +-
 .../flink/table/expressions/RexNodeConverter.java  |  8 ++--
 .../aggfunctions/ConcatWithRetractAggFunction.java |  2 +-
 .../ConcatWsWithRetractAggFunction.java            |  2 +-
 .../aggfunctions/FirstValueAggFunction.java        |  8 ++--
 .../FirstValueWithRetractAggFunction.java          | 12 +++---
 .../aggfunctions/LastValueAggFunction.java         |  8 ++--
 .../LastValueWithRetractAggFunction.java           | 12 +++---
 .../functions/aggfunctions/LeadLagAggFunction.java |  2 +-
 .../aggfunctions/MaxWithRetractAggFunction.java    |  4 +-
 .../aggfunctions/MinWithRetractAggFunction.java    |  4 +-
 .../table/operations/PlannerQueryOperation.java    |  2 +-
 .../apache/flink/table/plan/util/JoinTypeUtil.java |  2 +-
 .../flink/table/plan/util/KeySelectorUtil.java     |  4 +-
 .../table/sqlexec/SqlToOperationConverter.java     |  2 +-
 .../table/calcite/FlinkLogicalRelFactories.scala   |  2 +-
 .../flink/table/calcite/FlinkRelBuilder.scala      |  2 +-
 .../flink/table/calcite/FlinkRelFactories.scala    |  2 +-
 .../flink/table/calcite/FlinkTypeSystem.scala      |  2 +-
 .../flink/table/codegen/CalcCodeGenerator.scala    | 11 +++---
 .../apache/flink/table/codegen/CodeGenUtils.scala  | 19 ++++-----
 .../flink/table/codegen/CodeGeneratorContext.scala |  6 +--
 .../table/codegen/CollectorCodeGenerator.scala     |  2 +-
 .../table/codegen/CorrelateCodeGenerator.scala     | 11 +++---
 .../table/codegen/EqualiserCodeGenerator.scala     |  4 +-
 .../flink/table/codegen/ExpandCodeGenerator.scala  |  2 +-
 .../flink/table/codegen/ExprCodeGenerator.scala    |  9 +++--
 .../table/codegen/FunctionCodeGenerator.scala      |  2 +-
 .../apache/flink/table/codegen/GenerateUtils.scala |  7 ++--
 .../flink/table/codegen/GeneratedExpression.scala  |  2 +-
 .../flink/table/codegen/HashCodeGenerator.scala    |  2 +-
 .../table/codegen/InputFormatCodeGenerator.scala   |  2 +-
 .../table/codegen/LongHashJoinGenerator.scala      | 10 ++---
 .../table/codegen/LookupJoinCodeGenerator.scala    |  6 +--
 .../flink/table/codegen/MatchCodeGenerator.scala   |  6 +--
 .../codegen/NestedLoopJoinCodeGenerator.scala      |  6 +--
 .../table/codegen/OperatorCodeGenerator.scala      |  2 +-
 .../table/codegen/ProjectionCodeGenerator.scala    |  2 +-
 .../flink/table/codegen/SinkCodeGenerator.scala    |  7 ++--
 .../flink/table/codegen/ValuesCodeGenerator.scala  |  4 +-
 .../codegen/agg/AggsHandlerCodeGenerator.scala     | 11 +++---
 .../table/codegen/agg/DeclarativeAggCodeGen.scala  |  2 +-
 .../table/codegen/agg/DistinctAggCodeGen.scala     |  2 +-
 .../table/codegen/agg/ImperativeAggCodeGen.scala   |  7 ++--
 .../table/codegen/agg/batch/AggCodeGenHelper.scala | 14 ++++---
 .../agg/batch/AggWithoutKeysCodeGenerator.scala    |  6 +--
 .../codegen/agg/batch/HashAggCodeGenHelper.scala   |  8 ++--
 .../codegen/agg/batch/HashAggCodeGenerator.scala   |  6 +--
 .../agg/batch/HashWindowCodeGenerator.scala        | 14 +++----
 .../codegen/agg/batch/SortAggCodeGenerator.scala   |  4 +-
 .../agg/batch/SortWindowCodeGenerator.scala        |  6 +--
 .../codegen/agg/batch/WindowCodeGenerator.scala    |  6 +--
 .../flink/table/codegen/calls/DivCallGen.scala     |  2 +-
 .../table/codegen/calls/FunctionGenerator.scala    |  2 +-
 .../flink/table/codegen/calls/PrintCallGen.scala   |  2 +-
 .../codegen/calls/ScalarFunctionCallGen.scala      |  2 +-
 .../table/codegen/calls/ScalarOperatorGens.scala   | 11 +++---
 .../flink/table/codegen/calls/StringCallGen.scala  |  2 +-
 .../table/codegen/calls/TableFunctionCallGen.scala |  2 +-
 ...ltiFieldRangeBoundComparatorCodeGenerator.scala |  2 +-
 .../over/RangeBoundComparatorCodeGenerator.scala   |  2 +-
 .../codegen/sort/ComparatorCodeGenerator.scala     |  2 +-
 .../table/codegen/sort/SortCodeGenerator.scala     |  6 +--
 .../flink/table/dataview/DataViewUtils.scala       |  3 +-
 .../expressions/PlannerExpressionConverter.scala   |  4 +-
 .../flink/table/expressions/Reinterpret.scala      |  2 +-
 .../table/expressions/ReturnTypeInference.scala    |  5 ++-
 .../flink/table/expressions/aggregations.scala     |  2 +-
 .../flink/table/expressions/arithmetic.scala       |  2 +-
 .../org/apache/flink/table/expressions/call.scala  |  6 +--
 .../org/apache/flink/table/expressions/cast.scala  |  2 +-
 .../flink/table/expressions/fieldExpression.scala  |  2 +-
 .../table/functions/utils/AggSqlFunction.scala     |  2 +-
 .../table/functions/utils/ScalarSqlFunction.scala  |  6 +--
 .../table/functions/utils/TableSqlFunction.scala   |  2 +-
 .../functions/utils/UserDefinedFunctionUtils.scala | 12 +++---
 .../plan/metadata/FlinkRelMdColumnInterval.scala   |  2 +-
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |  2 +-
 .../table/plan/metadata/FlinkRelMdUniqueKeys.scala |  2 +-
 .../table/plan/nodes/calcite/LogicalRank.scala     |  2 +-
 .../flink/table/plan/nodes/calcite/Rank.scala      |  2 +-
 .../flink/table/plan/nodes/calcite/Sink.scala      |  2 +-
 .../table/plan/nodes/common/CommonLookupJoin.scala | 12 +++---
 .../plan/nodes/common/CommonPhysicalJoin.scala     |  2 +-
 .../plan/nodes/logical/FlinkLogicalRank.scala      |  2 +-
 .../plan/nodes/physical/batch/BatchExecCalc.scala  |  2 +-
 .../nodes/physical/batch/BatchExecExchange.scala   |  4 +-
 .../nodes/physical/batch/BatchExecExpand.scala     |  2 +-
 .../batch/BatchExecHashAggregateBase.scala         |  4 +-
 .../nodes/physical/batch/BatchExecHashJoin.scala   |  4 +-
 .../batch/BatchExecHashWindowAggregateBase.scala   |  6 +--
 .../nodes/physical/batch/BatchExecJoinBase.scala   |  2 +-
 .../plan/nodes/physical/batch/BatchExecLimit.scala |  2 +-
 .../physical/batch/BatchExecNestedLoopJoin.scala   |  2 +-
 .../physical/batch/BatchExecOverAggregate.scala    | 10 ++---
 .../plan/nodes/physical/batch/BatchExecRank.scala  |  6 +--
 .../plan/nodes/physical/batch/BatchExecSink.scala  |  5 ++-
 .../plan/nodes/physical/batch/BatchExecSort.scala  |  4 +-
 .../batch/BatchExecSortAggregateBase.scala         |  4 +-
 .../nodes/physical/batch/BatchExecSortLimit.scala  |  4 +-
 .../physical/batch/BatchExecSortMergeJoin.scala    |  4 +-
 .../batch/BatchExecSortWindowAggregateBase.scala   |  4 +-
 .../nodes/physical/stream/StreamExecCalc.scala     |  4 +-
 .../physical/stream/StreamExecCorrelate.scala      |  2 +-
 .../physical/stream/StreamExecDataStreamScan.scala |  6 +--
 .../physical/stream/StreamExecDeduplicate.scala    |  6 +--
 .../nodes/physical/stream/StreamExecExchange.scala |  2 +-
 .../nodes/physical/stream/StreamExecExpand.scala   |  2 +-
 .../stream/StreamExecGlobalGroupAggregate.scala    | 10 ++---
 .../physical/stream/StreamExecGroupAggregate.scala |  8 ++--
 .../stream/StreamExecGroupWindowAggregate.scala    |  9 ++---
 .../StreamExecIncrementalGroupAggregate.scala      |  8 ++--
 .../nodes/physical/stream/StreamExecJoin.scala     |  8 ++--
 .../nodes/physical/stream/StreamExecLimit.scala    |  4 +-
 .../stream/StreamExecLocalGroupAggregate.scala     |  6 +--
 .../nodes/physical/stream/StreamExecMatch.scala    |  4 +-
 .../physical/stream/StreamExecOverAggregate.scala  |  6 +--
 .../nodes/physical/stream/StreamExecRank.scala     |  4 +-
 .../nodes/physical/stream/StreamExecSink.scala     |  2 +-
 .../nodes/physical/stream/StreamExecSort.scala     |  4 +-
 .../physical/stream/StreamExecSortLimit.scala      |  4 +-
 .../stream/StreamExecTableSourceScan.scala         |  2 +-
 .../physical/stream/StreamExecTemporalJoin.scala   |  6 +--
 .../physical/stream/StreamExecTemporalSort.scala   |  4 +-
 .../stream/StreamExecWatermarkAssigner.scala       |  4 +-
 .../physical/stream/StreamExecWindowJoin.scala     |  6 +--
 .../logical/BatchLogicalWindowAggregateRule.scala  |  2 +-
 .../plan/rules/logical/CalcRankTransposeRule.scala |  2 +-
 .../plan/rules/logical/FlinkLogicalRankRule.scala  |  2 +-
 .../plan/rules/logical/LogicalUnnestRule.scala     |  4 +-
 .../logical/LogicalWindowAggregateRuleBase.scala   |  2 +-
 .../rules/logical/RankNumberColumnRemoveRule.scala |  2 +-
 .../logical/StreamLogicalWindowAggregateRule.scala |  2 +-
 .../physical/batch/BatchExecAggRuleBase.scala      |  2 +-
 .../physical/batch/BatchExecHashAggRule.scala      |  2 +-
 .../physical/batch/BatchExecLookupJoinRule.scala   |  2 +-
 .../rules/physical/batch/BatchExecRankRule.scala   |  2 +-
 .../physical/batch/BatchExecSortAggRule.scala      |  2 +-
 .../batch/BatchExecWindowAggregateRule.scala       |  2 +-
 .../stream/StreamExecDeduplicateRule.scala         |  2 +-
 .../physical/stream/StreamExecLookupJoinRule.scala |  2 +-
 .../schema/DeferredTypeFlinkTableFunction.scala    |  2 +-
 .../flink/table/plan/schema/InlineTable.scala      |  2 +-
 .../flink/table/plan/schema/TableSinkTable.scala   |  2 +-
 .../plan/schema/TypedFlinkTableFunction.scala      |  4 +-
 .../flink/table/plan/util/AggFunctionFactory.scala |  4 +-
 .../flink/table/plan/util/AggregateUtil.scala      |  8 ++--
 .../flink/table/plan/util/FlinkRelMdUtil.scala     |  6 +--
 .../apache/flink/table/plan/util/JoinUtil.scala    |  2 +-
 .../flink/table/plan/util/PartitionPruner.scala    |  2 +-
 .../apache/flink/table/plan/util/RankUtil.scala    |  2 +-
 .../flink/table/plan/util/RexNodeExtractor.scala   |  2 +-
 .../apache/flink/table/plan/util/ScanUtil.scala    |  8 ++--
 .../flink/table/plan/util/SetOpRewriteUtil.scala   |  2 +-
 .../flink/table/plan/util/WindowEmitStrategy.scala |  4 +-
 .../flink/table/plan/util/WindowJoinUtil.scala     |  2 +-
 .../flink/table/sinks/DataStreamTableSink.scala    |  2 +-
 .../apache/flink/table/sinks/TableSinkUtils.scala  |  8 ++--
 .../flink/table/sources/TableSourceUtil.scala      |  4 +-
 .../flink/table/typeutils/TypeCoercion.scala       |  2 +-
 .../table/codegen/LongHashJoinGeneratorTest.java   |  8 ++--
 .../flink/table/codegen/SortCodeGeneratorTest.java | 18 ++++-----
 .../FirstValueAggFunctionWithOrderTest.java        |  2 +-
 .../FirstValueAggFunctionWithoutOrderTest.java     |  2 +-
 ...stValueWithRetractAggFunctionWithOrderTest.java |  2 +-
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  2 +-
 .../LastValueAggFunctionWithOrderTest.java         |  2 +-
 .../LastValueAggFunctionWithoutOrderTest.java      |  2 +-
 ...stValueWithRetractAggFunctionWithOrderTest.java |  2 +-
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  2 +-
 .../MaxWithRetractAggFunctionTest.java             |  2 +-
 .../MinWithRetractAggFunctionTest.java             |  2 +-
 .../apache/flink/table/util/BaseRowTestUtil.java   |  4 +-
 .../codegen/ProjectionCodeGeneratorTest.scala      |  2 +-
 .../codegen/agg/AggsHandlerCodeGeneratorTest.scala |  4 +-
 .../codegen/agg/batch/AggWithoutKeysTest.scala     |  4 +-
 .../table/codegen/agg/batch/BatchAggTestBase.scala |  4 +-
 .../agg/batch/HashAggCodeGeneratorTest.scala       |  2 +-
 .../agg/batch/SortAggCodeGeneratorTest.scala       |  4 +-
 .../flink/table/expressions/DecimalTypeTest.scala  |  3 +-
 .../expressions/utils/ExpressionTestBase.scala     |  2 +-
 .../table/expressions/utils/RowTypeTestBase.scala  |  2 +-
 .../utils/ScalarOperatorsTestBase.scala            |  2 +-
 .../expressions/utils/ScalarTypesTestBase.scala    |  3 +-
 .../table/plan/batch/sql/TableSourceTest.scala     |  5 +--
 .../plan/batch/sql/agg/AggregateTestBase.scala     |  2 +-
 .../metadata/FlinkRelMdColumnUniquenessTest.scala  |  2 +-
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  |  2 +-
 .../FlinkRelMdModifiedMonotonicityTest.scala       |  2 +-
 .../table/plan/metadata/MetadataTestUtil.scala     |  4 +-
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  4 +-
 .../PushProjectIntoTableSourceScanRuleTest.scala   |  2 +-
 .../table/plan/stream/sql/agg/AggregateTest.scala  |  2 +-
 .../plan/stream/sql/join/LookupJoinTest.scala      |  2 +-
 .../flink/table/plan/util/RexNodeTestBase.scala    |  2 +-
 .../table/runtime/batch/sql/CorrelateITCase.scala  |  2 +-
 .../table/runtime/batch/sql/DecimalITCase.scala    |  6 +--
 .../runtime/batch/sql/TableSourceITCase.scala      |  2 +-
 .../table/runtime/batch/sql/UnionITCase.scala      |  2 +-
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  2 +-
 .../table/runtime/batch/sql/join/JoinITCase.scala  |  2 +-
 .../table/runtime/batch/table/DecimalITCase.scala  |  6 +--
 ...AbstractTwoInputStreamOperatorWithTTLTest.scala |  9 ++---
 .../table/runtime/stream/sql/AggregateITCase.scala |  4 +-
 .../table/runtime/stream/sql/CalcITCase.scala      |  2 +-
 .../table/runtime/stream/sql/RankITCase.scala      |  2 +-
 .../table/runtime/stream/sql/ValuesITCase.scala    |  2 +-
 .../flink/table/runtime/utils/BatchTestBase.scala  |  2 +-
 .../flink/table/runtime/utils/StreamTestSink.scala |  4 +-
 .../runtime/utils/StreamingWithStateTestBase.scala |  4 +-
 .../flink/table/runtime/utils/TableUtil.scala      |  2 +-
 .../flink/table/runtime/utils/TestSinkUtil.scala   |  4 +-
 .../apache/flink/table/util/AvgAggFunction.scala   |  2 +-
 .../apache/flink/table/util/TableTestBase.scala    |  2 +-
 .../table/dataformat/AbstractBinaryWriter.java     |  8 ++--
 .../apache/flink/table/dataformat/BinaryArray.java |  2 +-
 .../flink/table/dataformat/BinaryArrayWriter.java  |  2 +-
 .../flink/table/dataformat/BinaryFormat.java       |  2 +-
 .../flink/table/dataformat/BinaryGeneric.java      |  2 +-
 .../apache/flink/table/dataformat/BinaryMap.java   |  4 +-
 .../apache/flink/table/dataformat/BinaryRow.java   |  2 +-
 .../flink/table/dataformat/BinaryRowWriter.java    |  2 +-
 .../flink/table/dataformat/BinaryString.java       |  4 +-
 .../flink/table/dataformat/BinaryStringUtil.java   |  2 +-
 .../flink/table/dataformat/BinaryWriter.java       |  6 +--
 .../table/dataformat/DataFormatConverters.java     | 10 ++---
 .../org/apache/flink/table/dataformat/Decimal.java |  4 +-
 .../apache/flink/table/dataformat/NestedRow.java   |  2 +-
 .../dataview/NullAwareMapIterator.java             |  2 +-
 .../dataview/PerKeyStateDataViewStore.java         |  4 +-
 .../dataview/PerWindowStateDataViewStore.java      |  4 +-
 .../{ => runtime}/dataview/StateDataView.java      |  2 +-
 .../{ => runtime}/dataview/StateDataViewStore.java |  4 +-
 .../{ => runtime}/dataview/StateListView.java      |  2 +-
 .../table/{ => runtime}/dataview/StateMapView.java |  2 +-
 .../runtime/functions/SqlLikeChainChecker.java     |  2 +-
 .../generated/AggsHandleFunction.java              |  4 +-
 .../{ => runtime}/generated/CompileUtils.java      |  2 +-
 .../generated/GeneratedAggsHandleFunction.java     |  2 +-
 .../{ => runtime}/generated/GeneratedClass.java    |  2 +-
 .../generated/GeneratedCollector.java              |  2 +-
 .../{ => runtime}/generated/GeneratedFunction.java |  2 +-
 .../generated/GeneratedHashFunction.java           |  2 +-
 .../{ => runtime}/generated/GeneratedInput.java    |  2 +-
 .../generated/GeneratedJoinCondition.java          |  2 +-
 .../GeneratedNamespaceAggsHandleFunction.java      |  2 +-
 .../generated/GeneratedNormalizedKeyComputer.java  |  2 +-
 .../{ => runtime}/generated/GeneratedOperator.java |  2 +-
 .../generated/GeneratedProjection.java             |  2 +-
 .../generated/GeneratedRecordComparator.java       |  2 +-
 .../generated/GeneratedRecordEqualiser.java        |  2 +-
 .../generated/GeneratedResultFuture.java           |  2 +-
 .../{ => runtime}/generated/HashFunction.java      |  2 +-
 .../{ => runtime}/generated/JoinCondition.java     |  2 +-
 .../generated/NamespaceAggsHandleFunction.java     |  4 +-
 .../generated/NormalizedKeyComputer.java           |  4 +-
 .../table/{ => runtime}/generated/Projection.java  |  2 +-
 .../{ => runtime}/generated/RecordComparator.java  |  4 +-
 .../{ => runtime}/generated/RecordEqualiser.java   |  2 +-
 .../runtime/hashtable/BinaryHashPartition.java     |  2 +-
 .../table/runtime/hashtable/BinaryHashTable.java   | 12 +++---
 .../table/runtime/hashtable/BuildSideIterator.java |  2 +-
 .../table/runtime/hashtable/LongHashPartition.java |  2 +-
 .../runtime/hashtable/LongHybridHashTable.java     |  2 +-
 .../io/BinaryRowChannelInputViewIterator.java      |  2 +-
 .../runtime/keyselector/BaseRowKeySelector.java    |  2 +-
 .../runtime/keyselector/BinaryRowKeySelector.java  |  6 +--
 .../keyselector/NullBinaryRowKeySelector.java      |  2 +-
 .../AbstractProcessStreamOperator.java             |  2 +-
 .../{ => operators}/CodeGenOperatorFactory.java    | 19 ++++-----
 .../{ => operators}/TableStreamOperator.java       |  2 +-
 .../{ => operators}/aggregate/BytesHashMap.java    |  4 +-
 .../BytesHashMapSpillMemorySegmentPool.java        |  2 +-
 .../aggregate/GroupAggFunction.java                | 14 +++----
 .../aggregate/MiniBatchGlobalGroupAggFunction.java | 16 ++++----
 .../aggregate/MiniBatchGroupAggFunction.java       | 18 ++++-----
 .../MiniBatchIncrementalGroupAggFunction.java      | 10 ++---
 .../aggregate/MiniBatchLocalGroupAggFunction.java  | 10 ++---
 .../{ => operators}/aggregate/RecordCounter.java   |  2 +-
 .../bundle/AbstractMapBundleOperator.java          |  6 +--
 .../bundle/KeyedMapBundleOperator.java             |  4 +-
 .../{ => operators}/bundle/MapBundleFunction.java  |  2 +-
 .../{ => operators}/bundle/MapBundleOperator.java  |  4 +-
 .../bundle/trigger/BundleTrigger.java              |  2 +-
 .../bundle/trigger/BundleTriggerCallback.java      |  2 +-
 .../bundle/trigger/CoBundleTrigger.java            |  2 +-
 .../bundle/trigger/CountBundleTrigger.java         |  2 +-
 .../bundle/trigger/CountCoBundleTrigger.java       |  2 +-
 .../deduplicate/DeduplicateFunctionHelper.java     |  2 +-
 .../DeduplicateKeepFirstRowFunction.java           |  4 +-
 .../DeduplicateKeepLastRowFunction.java            |  6 +--
 .../MiniBatchDeduplicateKeepFirstRowFunction.java  |  6 +--
 .../MiniBatchDeduplicateKeepLastRowFunction.java   |  8 ++--
 .../{ => operators}/join/EmitAwareCollector.java   |  2 +-
 .../{ => operators}/join/FlinkJoinType.java        |  2 +-
 .../{ => operators}/join/HashJoinOperator.java     | 12 +++---
 .../runtime/{ => operators}/join/HashJoinType.java |  2 +-
 .../KeyedCoProcessOperatorWithWatermarkDelay.java  |  2 +-
 .../{ => operators}/join/NullAwareJoinHelper.java  |  2 +-
 .../{ => operators}/join/OuterJoinPaddingUtil.java |  2 +-
 .../join/ProcTimeBoundedStreamJoin.java            |  6 +--
 .../join/RowTimeBoundedStreamJoin.java             |  6 +--
 .../join/SortMergeFullOuterJoinIterator.java       |  8 ++--
 .../join/SortMergeInnerJoinIterator.java           |  8 ++--
 .../join/SortMergeJoinIterator.java                |  8 ++--
 .../join/SortMergeJoinOperator.java                | 24 ++++++------
 .../join/SortMergeOneSideOuterJoinIterator.java    |  8 ++--
 .../join/TimeBoundedStreamJoin.java                |  6 +--
 .../join/lookup/AsyncLookupJoinRunner.java         |  8 ++--
 .../join/lookup/AsyncLookupJoinWithCalcRunner.java |  8 ++--
 .../join/lookup/DelegatingResultFuture.java        |  2 +-
 .../join/lookup/LookupJoinRunner.java              |  6 +--
 .../join/lookup/LookupJoinWithCalcRunner.java      |  6 +--
 .../join/stream/AbstractStreamingJoinOperator.java | 16 ++++----
 .../join/stream/StreamingJoinOperator.java         | 16 ++++----
 .../join/stream/StreamingSemiAntiJoinOperator.java | 16 ++++----
 .../join/stream/state/JoinInputSideSpec.java       |  4 +-
 .../join/stream/state/JoinRecordStateView.java     |  2 +-
 .../join/stream/state/JoinRecordStateViews.java    |  4 +-
 .../stream/state/OuterJoinRecordStateView.java     |  2 +-
 .../stream/state/OuterJoinRecordStateViews.java    |  6 +--
 ...seTwoInputStreamOperatorWithStateRetention.java |  2 +-
 .../temporal/TemporalProcessTimeJoinOperator.java  |  8 ++--
 .../join/temporal/TemporalRowTimeJoinOperator.java |  8 ++--
 .../match/BaseRowEventComparator.java              |  6 +--
 .../match/IterativeConditionRunner.java            |  4 +-
 .../match/PatternProcessFunctionRunner.java        |  4 +-
 .../match/RowtimeProcessFunction.java              |  2 +-
 .../AbstractRowTimeUnboundedPrecedingOver.java     | 10 ++---
 .../over/BufferDataOverWindowOperator.java         | 12 +++---
 .../over/NonBufferOverWindowOperator.java          | 16 ++++----
 .../ProcTimeRangeBoundedPrecedingFunction.java     | 10 ++---
 .../over/ProcTimeRowsBoundedPrecedingFunction.java | 10 ++---
 .../over/ProcTimeUnboundedPrecedingFunction.java   | 10 ++---
 .../over/RowTimeRangeBoundedPrecedingFunction.java | 10 ++---
 .../RowTimeRangeUnboundedPrecedingFunction.java    |  4 +-
 .../over/RowTimeRowsBoundedPrecedingFunction.java  | 10 ++---
 .../RowTimeRowsUnboundedPrecedingFunction.java     |  4 +-
 .../over/frame/InsensitiveOverFrame.java           |  8 ++--
 .../over/frame/OffsetOverFrame.java                |  8 ++--
 .../over/frame/OverWindowFrame.java                |  2 +-
 .../over/frame/RangeSlidingOverFrame.java          |  8 ++--
 .../frame/RangeUnboundedFollowingOverFrame.java    |  8 ++--
 .../frame/RangeUnboundedPrecedingOverFrame.java    |  8 ++--
 .../over/frame/RowSlidingOverFrame.java            |  4 +-
 .../over/frame/RowUnboundedFollowingOverFrame.java |  4 +-
 .../over/frame/RowUnboundedPrecedingOverFrame.java |  4 +-
 .../over/frame/SlidingOverFrame.java               | 10 ++---
 .../over/frame/UnboundedFollowingOverFrame.java    | 10 ++---
 .../over/frame/UnboundedOverWindowFrame.java       | 10 ++---
 .../over/frame/UnboundedPrecedingOverFrame.java    |  8 ++--
 .../{ => operators}/rank/AbstractTopNFunction.java |  6 +--
 .../rank/AppendOnlyTopNFunction.java               |  6 +--
 .../{ => operators}/rank/ConstantRankRange.java    |  2 +-
 .../rank/ConstantRankRangeWithoutEnd.java          |  2 +-
 .../runtime/{ => operators}/rank/RankRange.java    |  2 +-
 .../runtime/{ => operators}/rank/RankType.java     |  2 +-
 .../rank/RetractableTopNFunction.java              | 12 +++---
 .../runtime/{ => operators}/rank/TopNBuffer.java   |  2 +-
 .../rank/UpdatableTopNFunction.java                |  6 +--
 .../{ => operators}/rank/VariableRankRange.java    |  2 +-
 .../sort/AbstractBinaryExternalMerger.java         |  2 +-
 .../sort/BaseTemporalSortOperator.java             |  2 +-
 .../{ => operators}/sort/BinaryExternalMerger.java |  6 +--
 .../{ => operators}/sort/BinaryExternalSorter.java | 10 ++---
 .../sort/BinaryInMemorySortBuffer.java             | 10 ++---
 .../sort/BinaryIndexedSortable.java                |  8 ++--
 .../sort/BinaryKVExternalMerger.java               |  6 +--
 .../sort/BinaryKVInMemorySortBuffer.java           |  8 ++--
 .../{ => operators}/sort/BinaryMergeIterator.java  |  4 +-
 .../sort/BufferedKVExternalSorter.java             |  8 ++--
 .../sort/ChannelReaderKVInputViewIterator.java     |  2 +-
 .../{ => operators}/sort/LimitOperator.java        |  4 +-
 .../sort/ListMemorySegmentPool.java                |  2 +-
 .../{ => operators}/sort/ProcTimeSortOperator.java |  8 ++--
 .../runtime/{ => operators}/sort/RankOperator.java | 10 ++---
 .../{ => operators}/sort/RowTimeSortOperator.java  |  8 ++--
 .../{ => operators}/sort/SortLimitOperator.java    | 10 ++---
 .../runtime/{ => operators}/sort/SortOperator.java | 16 ++++----
 .../runtime/{ => operators}/sort/SortUtil.java     |  2 +-
 .../{ => operators}/sort/SpillChannelManager.java  |  2 +-
 .../{ => operators}/sort/StreamSortOperator.java   | 12 +++---
 .../{ => operators}/values/ValuesInputFormat.java  |  6 +--
 .../operators}/window/CountWindow.java             |  2 +-
 .../operators}/window/TimeWindow.java              |  2 +-
 .../{api => runtime/operators}/window/Window.java  |  2 +-
 .../{ => operators}/window/WindowOperator.java     | 33 ++++++++--------
 .../window/WindowOperatorBuilder.java              | 32 +++++++--------
 .../assigners/CountSlidingWindowAssigner.java      |  6 +--
 .../assigners/CountTumblingWindowAssigner.java     |  6 +--
 .../assigners/InternalTimeWindowAssigner.java      |  2 +-
 .../window/assigners/MergingWindowAssigner.java    |  4 +-
 .../window/assigners/PanedWindowAssigner.java      |  4 +-
 .../window/assigners/SessionWindowAssigner.java    |  4 +-
 .../window/assigners/SlidingWindowAssigner.java    |  4 +-
 .../window/assigners/TumblingWindowAssigner.java   |  4 +-
 .../window/assigners/WindowAssigner.java           |  8 ++--
 .../window/grouping/HeapWindowsGrouping.java       |  2 +-
 .../window/grouping/WindowsGrouping.java           |  4 +-
 .../internal/GeneralWindowProcessFunction.java     |  8 ++--
 .../internal/InternalWindowProcessFunction.java    | 10 ++---
 .../internal/MergingWindowProcessFunction.java     |  8 ++--
 .../window/internal/MergingWindowSet.java          |  6 +--
 .../internal/PanedWindowProcessFunction.java       |  8 ++--
 .../window/triggers/ElementTriggers.java           |  4 +-
 .../window/triggers/EventTimeTriggers.java         |  4 +-
 .../window/triggers/ProcessingTimeTriggers.java    |  4 +-
 .../{ => operators}/window/triggers/Trigger.java   |  6 +--
 .../wmassigners}/MiniBatchAssignerOperator.java    |  2 +-
 .../MiniBatchedWatermarkAssignerOperator.java      |  2 +-
 .../wmassigners}/WatermarkAssignerOperator.java    |  2 +-
 .../{ => partitioner}/BinaryHashPartitioner.java   |  6 +--
 .../types/ClassDataTypeConverter.java              |  3 +-
 .../types/ClassLogicalTypeConverter.java           |  2 +-
 .../{ => runtime}/types/InternalSerializers.java   | 14 +++----
 .../types/LogicalTypeDataTypeConverter.java        | 11 +++---
 .../{ => runtime}/types/PlannerTypeUtils.java      |  3 +-
 .../types/TypeInfoDataTypeConverter.java           | 18 +++++----
 .../types/TypeInfoLogicalTypeConverter.java        |  7 ++--
 .../typeutils/AbstractMapSerializer.java           |  2 +-
 .../typeutils/AbstractMapTypeInfo.java             |  2 +-
 .../typeutils/AbstractRowSerializer.java           |  2 +-
 .../typeutils/BaseArraySerializer.java             | 23 +++++------
 .../{ => runtime}/typeutils/BaseMapSerializer.java | 21 +++++-----
 .../{ => runtime}/typeutils/BaseRowSerializer.java |  4 +-
 .../{ => runtime}/typeutils/BaseRowTypeInfo.java   |  4 +-
 .../typeutils/BigDecimalTypeInfo.java              |  2 +-
 .../typeutils/BinaryGenericSerializer.java         |  4 +-
 .../typeutils/BinaryRowSerializer.java             |  4 +-
 .../typeutils/BinaryStringSerializer.java          |  4 +-
 .../typeutils/BinaryStringTypeInfo.java            |  2 +-
 .../typeutils/BinaryStringTypeInfoFactory.java     |  2 +-
 .../{ => runtime}/typeutils/DecimalSerializer.java |  2 +-
 .../{ => runtime}/typeutils/DecimalTypeInfo.java   |  2 +-
 .../typeutils/DecimalTypeInfoFactory.java          |  2 +-
 .../typeutils/SortedMapSerializer.java             |  2 +-
 .../typeutils/SortedMapSerializerSnapshot.java     |  2 +-
 .../{ => runtime}/typeutils/SortedMapTypeInfo.java |  2 +-
 .../{ => runtime}/typeutils/TypeCheckUtils.java    |  2 +-
 .../table/{ => runtime}/util/MurmurHashUtil.java   | 17 ++++----
 .../runtime/util/ResettableExternalBuffer.java     |  4 +-
 .../table/{ => runtime}/util/SegmentsUtil.java     | 17 ++++----
 .../flink/table/runtime/util/StringUtf8Utils.java  |  4 +-
 .../runtime/util/collections/DoubleHashSet.java    |  2 +-
 .../runtime/util/collections/FloatHashSet.java     |  2 +-
 .../table/runtime/util/collections/IntHashSet.java |  2 +-
 .../runtime/util/collections/LongHashSet.java      |  2 +-
 .../runtime/util/collections/ShortHashSet.java     |  2 +-
 .../apache/flink/table/dataformat/BaseRowTest.java |  6 +--
 .../flink/table/dataformat/BinaryArrayTest.java    |  8 ++--
 .../flink/table/dataformat/BinaryRowTest.java      |  8 ++--
 .../flink/table/dataformat/BinaryStringTest.java   |  2 +-
 .../table/dataformat/DataFormatConvertersTest.java |  6 +--
 .../flink/table/dataformat/NestedRowTest.java      |  2 +-
 .../{ => runtime}/generated/CompileUtilsTest.java  |  2 +-
 .../generated/GeneratedCollectorWrapper.java       |  2 +-
 .../generated/GeneratedFunctionWrapper.java        |  2 +-
 .../generated/GeneratedResultFutureWrapper.java    |  2 +-
 .../runtime/hashtable/BinaryHashTableTest.java     |  6 +--
 .../table/runtime/hashtable/LongHashTableTest.java |  2 +-
 .../aggregate/BytesHashMapTest.java                |  4 +-
 .../{ => operators}/aggregate/HashAggTest.java     |  2 +-
 .../aggregate/SumHashAggTestOperator.java          | 10 ++---
 .../bundle/MapBundleOperatorTest.java              |  4 +-
 .../bundle/trigger/CountBundleTriggerTest.java     |  2 +-
 .../bundle/trigger/CountCoBundleTriggerTest.java   |  2 +-
 .../bundle/trigger/TestTriggerCallback.java        |  2 +-
 .../deduplicate/DeduplicateFunctionTestBase.java   |  4 +-
 .../DeduplicateKeepFirstRowFunctionTest.java       |  2 +-
 .../DeduplicateKeepLastRowFunctionTest.java        |  2 +-
 ...niBatchDeduplicateKeepFirstRowFunctionTest.java |  6 +--
 ...iniBatchDeduplicateKeepLastRowFunctionTest.java |  6 +--
 .../join/AsyncLookupJoinHarnessTest.java           | 18 ++++-----
 .../join/Int2HashJoinOperatorTest.java             | 14 +++----
 .../join/Int2SortMergeJoinOperatorTest.java        | 28 +++++++-------
 .../join/LookupJoinHarnessTest.java                | 12 +++---
 .../join/ProcTimeBoundedStreamJoinTest.java        |  4 +-
 .../join/RandomSortMergeInnerJoinTest.java         |  6 +--
 .../join/RandomSortMergeOuterJoinTest.java         |  4 +-
 .../join/RowTimeBoundedStreamJoinTest.java         |  4 +-
 .../join/SortMergeJoinIteratorTest.java            | 10 ++---
 .../join/String2HashJoinOperatorTest.java          | 14 +++----
 .../join/String2SortMergeJoinOperatorTest.java     | 45 +++++++++++-----------
 .../join/TimeBoundedStreamJoinTestBase.java        |  6 +--
 .../over/BufferDataOverWindowOperatorTest.java     | 34 ++++++++--------
 .../over/NonBufferOverWindowOperatorTest.java      | 14 +++----
 .../over/SumAggsHandleFunction.java                |  6 +--
 .../rank/AppendOnlyTopNFunctionTest.java           |  2 +-
 .../rank/RetractableTopNFunctionTest.java          |  2 +-
 .../{ => operators}/rank/TopNFunctionTestBase.java | 14 +++----
 .../rank/UpdatableTopNFunctionTest.java            |  2 +-
 .../sort/BinaryExternalSorterTest.java             |  6 +--
 .../sort/BinaryMergeIteratorTest.java              |  6 +--
 .../sort/BufferedKVExternalSorterTest.java         |  8 ++--
 .../sort/IntNormalizedKeyComputer.java             |  4 +-
 .../{ => operators}/sort/IntRecordComparator.java  |  4 +-
 .../sort/ProcTimeSortOperatorTest.java             |  8 ++--
 .../sort/RowTimeSortOperatorTest.java              |  8 ++--
 .../runtime/{ => operators}/sort/SortUtilTest.java |  2 +-
 .../sort/StreamSortOperatorTest.java               |  8 ++--
 .../sort/StringNormalizedKeyComputer.java          |  4 +-
 .../sort/StringRecordComparator.java               |  4 +-
 .../sort/TestMemorySegmentPool.java                |  2 +-
 .../window/MergingWindowSetTest.java               |  9 ++---
 .../window/WindowOperatorContractTest.java         | 14 +++----
 .../{ => operators}/window/WindowOperatorTest.java | 25 ++++++------
 .../{ => operators}/window/WindowTestUtils.java    |  4 +-
 .../assigners/SessionWindowAssignerTest.java       |  6 +--
 .../assigners/SlidingWindowAssignerTest.java       |  6 +--
 .../assigners/TumblingWindowAssignerTest.java      |  6 +--
 .../window/grouping/HeapWindowsGroupingTest.java   |  5 ++-
 .../window/triggers/TriggersTest.java              |  2 +-
 .../MiniBatchAssignerOperatorTest.java             |  2 +-
 .../MiniBatchedWatermarkAssignerOperatorTest.java  |  2 +-
 .../WatermarkAssignerOperatorTest.java             |  2 +-
 .../WatermarkAssignerOperatorTestBase.java         |  2 +-
 .../typeutils/BaseArraySerializerTest.java         |  8 ++--
 .../typeutils/BaseMapSerializerTest.java           |  8 ++--
 .../typeutils/BaseRowSerializerTest.java           |  2 +-
 .../typeutils/BigDecimalTypeInfoTest.java          |  2 +-
 .../typeutils/BinaryGenericSerializerTest.java     |  2 +-
 .../typeutils/BinaryRowSerializerTest.java         |  2 +-
 .../typeutils/BinaryRowTypeInfoTest.java           |  2 +-
 .../typeutils/BinaryStringSerializerTest.java      |  2 +-
 .../typeutils/DecimalSerializerTest.java           |  2 +-
 .../typeutils/InternalTypeInfoTest.java            |  2 +-
 .../typeutils/SerializerTestUtil.java              |  4 +-
 .../table/runtime/util/BaseRowHarnessAssertor.java |  2 +-
 .../table/runtime/util/BaseRowRecordEqualiser.java |  2 +-
 .../table/runtime/util/BinaryRowKeySelector.java   |  4 +-
 .../runtime/util/ResettableExternalBufferTest.java |  2 +-
 .../table/{ => runtime}/util/SegmentsUtilTest.java | 17 ++++----
 534 files changed, 1371 insertions(+), 1347 deletions(-)

diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
index 7053f3f..4a96b7a 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
@@ -126,7 +126,7 @@ public class ExecutionConfigOptions {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * See {@code org.apache.flink.table.runtime.window.grouping.HeapWindowsGrouping}.
+	 * See {@code org.apache.flink.table.runtime.operators.window.grouping.HeapWindowsGrouping}.
 	 */
 	public static final ConfigOption<Integer> SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT =
 			key("sql.exec.window-agg.buffer-size-limit")
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
index 5835397..06b0eaa 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
@@ -24,7 +24,7 @@ import org.apache.flink.table.calcite.FlinkRelBuilder;
 import org.apache.flink.table.calcite.FlinkTypeFactory;
 import org.apache.flink.table.plan.schema.FlinkTable;
 import org.apache.flink.table.plan.stats.FlinkStatistic;
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.util.JavaScalaConversionUtil;
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
index 34dc3bb..72ddfa3 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
@@ -33,7 +33,7 @@ import java.util.Optional;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
 import static org.apache.flink.table.util.JavaScalaConversionUtil.toJava;
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
index 5dfeb97..5de269f 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
@@ -93,11 +93,11 @@ import java.util.stream.Collectors;
 
 import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
 import static org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalType;
-import static org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
+import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
+import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isCharacterString;
+import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTemporal;
+import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTimeInterval;
 import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
-import static org.apache.flink.table.typeutils.TypeCheckUtils.isCharacterString;
-import static org.apache.flink.table.typeutils.TypeCheckUtils.isTemporal;
-import static org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval;
 
 /**
  * Visit expression to generator {@link RexNode}.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
index 8fa952d..8d2c7ca 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
@@ -23,7 +23,7 @@ import org.apache.flink.table.api.dataview.ListView;
 import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.BinaryStringUtil;
 import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
 import org.apache.flink.util.FlinkRuntimeException;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
index e0a7ee7..23cc4cc 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
@@ -23,7 +23,7 @@ import org.apache.flink.table.api.dataview.ListView;
 import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.BinaryStringUtil;
 import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
 import org.apache.flink.util.FlinkRuntimeException;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
index 7c6af1f..a4fa317 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
@@ -24,13 +24,13 @@ import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 
 /**
  * built-in FirstValue aggregate function.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
index 78cb668..799815a 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
@@ -39,20 +39,20 @@ import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataview.MapViewSerializer;
 import org.apache.flink.table.dataview.MapViewTypeInfo;
 import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalSerializer;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringSerializer;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalSerializer;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 
 /**
  * built-in FirstValue with retraction aggregate function.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
index 78aec59..5846735 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
@@ -24,13 +24,13 @@ import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 
 /**
  * built-in LastValue aggregate function.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
index f6df480..6455844 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
@@ -39,20 +39,20 @@ import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataview.MapViewSerializer;
 import org.apache.flink.table.dataview.MapViewTypeInfo;
 import org.apache.flink.table.functions.AggregateFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalSerializer;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringSerializer;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalSerializer;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 
 /**
  * built-in LastValue with retraction aggregate function.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
index 7b7c107..6e9c276 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.functions.aggfunctions;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.expressions.Expression;
 import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.runtime.over.frame.OffsetOverFrame;
+import org.apache.flink.table.runtime.operators.over.frame.OffsetOverFrame;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.TimeType;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
index 4c076f6..888cbfe 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
@@ -28,8 +28,8 @@ import org.apache.flink.table.api.dataview.MapView;
 import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.sql.Date;
 import java.sql.Time;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
index 3daaf45..f3c21bf 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
@@ -28,8 +28,8 @@ import org.apache.flink.table.api.dataview.MapView;
 import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.sql.Date;
 import java.sql.Time;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/PlannerQueryOperation.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/PlannerQueryOperation.java
index ee65ed6..663f79d 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/PlannerQueryOperation.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/operations/PlannerQueryOperation.java
@@ -21,8 +21,8 @@ package org.apache.flink.table.operations;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.calcite.FlinkTypeFactory;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
 import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter;
 
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/JoinTypeUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/JoinTypeUtil.java
index 33ad1fe..c6924de 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/JoinTypeUtil.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/JoinTypeUtil.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.util;
 
-import org.apache.flink.table.runtime.join.FlinkJoinType;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
 
 import org.apache.calcite.rel.core.JoinRelType;
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/KeySelectorUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/KeySelectorUtil.java
index 992861a..c8bbdeb 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/KeySelectorUtil.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/KeySelectorUtil.java
@@ -21,13 +21,13 @@ package org.apache.flink.table.plan.util;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.codegen.CodeGeneratorContext;
 import org.apache.flink.table.codegen.ProjectionCodeGenerator;
-import org.apache.flink.table.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector;
 import org.apache.flink.table.runtime.keyselector.BinaryRowKeySelector;
 import org.apache.flink.table.runtime.keyselector.NullBinaryRowKeySelector;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 
 /**
  * Utility for KeySelector.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java
index 25b01a4..e47c407 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java
@@ -35,7 +35,7 @@ import org.apache.flink.table.operations.Operation;
 import org.apache.flink.table.operations.PlannerQueryOperation;
 import org.apache.flink.table.operations.ddl.CreateTableOperation;
 import org.apache.flink.table.operations.ddl.DropTableOperation;
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
 
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLogicalRelFactories.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLogicalRelFactories.scala
index ce4bda2..992458c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLogicalRelFactories.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkLogicalRelFactories.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.calcite
 import org.apache.flink.table.calcite.FlinkRelFactories.{ExpandFactory, RankFactory, SinkFactory}
 import org.apache.flink.table.plan.nodes.logical._
 import org.apache.flink.table.plan.schema.FlinkRelOptTable
-import org.apache.flink.table.runtime.rank.{RankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType}
 import org.apache.flink.table.sinks.TableSink
 
 import com.google.common.collect.ImmutableList
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
index 77bb0f0..d4d6fe3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
@@ -27,7 +27,7 @@ import org.apache.flink.table.operations.QueryOperation
 import org.apache.flink.table.plan.QueryOperationConverter
 import org.apache.flink.table.plan.logical.LogicalWindow
 import org.apache.flink.table.plan.nodes.calcite.LogicalWindowAggregate
-import org.apache.flink.table.runtime.rank.{RankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType}
 import org.apache.flink.table.sinks.TableSink
 
 import org.apache.calcite.plan._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelFactories.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelFactories.scala
index 05dc497..2ef6042 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelFactories.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkRelFactories.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.calcite
 
 import org.apache.flink.table.plan.nodes.calcite.{LogicalExpand, LogicalRank, LogicalSink}
-import org.apache.flink.table.runtime.rank.{RankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType}
 import org.apache.flink.table.sinks.TableSink
 
 import org.apache.calcite.plan.Contexts
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
index 7f92345..4e6d716 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
@@ -18,8 +18,8 @@
 
 package org.apache.flink.table.calcite
 
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
 import org.apache.flink.table.types.logical.{DecimalType, DoubleType, LogicalType}
-import org.apache.flink.table.typeutils.TypeCheckUtils
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory, RelDataTypeSystemImpl}
 import org.apache.calcite.sql.`type`.SqlTypeName
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CalcCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CalcCodeGenerator.scala
index 22bc934..e9574be 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CalcCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CalcCodeGenerator.scala
@@ -17,16 +17,17 @@
  */
 package org.apache.flink.table.codegen
 
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function}
+import org.apache.flink.api.dag.Transformation
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.dataformat.{BaseRow, BoxedWrapperRow}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.generated.GeneratedFunction
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.RowType
+
 import org.apache.calcite.plan.RelOptCluster
 import org.apache.calcite.rex._
-import org.apache.flink.api.common.functions.{FlatMapFunction, Function}
-import org.apache.flink.api.dag.Transformation
-import org.apache.flink.table.generated.GeneratedFunction
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import scala.collection.JavaConversions._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
index 3bae0dd..193a4fb 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
@@ -22,18 +22,19 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.core.memory.MemorySegment
 import org.apache.flink.table.dataformat.DataFormatConverters.IdentityConverter
 import org.apache.flink.table.dataformat.util.BinaryRowUtil.BYTE_ARRAY_BASE_OFFSET
-import org.apache.flink.table.dataformat.{Decimal, BinaryStringUtil, _}
-import org.apache.flink.table.dataview.StateDataViewStore
+import org.apache.flink.table.dataformat.{BinaryStringUtil, Decimal, _}
 import org.apache.flink.table.functions.UserDefinedFunction
-import org.apache.flink.table.generated.{AggsHandleFunction, HashFunction, NamespaceAggsHandleFunction}
-import org.apache.flink.table.types.ClassLogicalTypeConverter.getInternalClassForType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.dataview.StateDataViewStore
+import org.apache.flink.table.runtime.generated.{AggsHandleFunction, HashFunction, NamespaceAggsHandleFunction}
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getInternalClassForType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
+import org.apache.flink.table.runtime.util.MurmurHashUtil
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.types.{ClassLogicalTypeConverter, DataType}
-import org.apache.flink.table.typeutils.TypeCheckUtils
-import org.apache.flink.table.util.MurmurHashUtil
 import org.apache.flink.types.Row
 
 import java.lang.reflect.Method
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGeneratorContext.scala
index 1570318..724fc1c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGeneratorContext.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CodeGeneratorContext.scala
@@ -24,11 +24,11 @@ import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.GenerateUtils.generateRecordStatement
-import org.apache.flink.table.dataformat.{DataFormatConverters, GenericRow}
+import org.apache.flink.table.dataformat.GenericRow
 import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction}
-import org.apache.flink.table.runtime.TableStreamOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
+import org.apache.flink.table.runtime.types.InternalSerializers
 import org.apache.flink.table.runtime.util.collections._
-import org.apache.flink.table.types.InternalSerializers
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
 import org.apache.flink.util.InstantiationUtil
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala
index 9b6e67a..e6f277e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.codegen
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.Indenter.toISC
-import org.apache.flink.table.generated.GeneratedCollector
 import org.apache.flink.table.runtime.collector.TableFunctionCollector
+import org.apache.flink.table.runtime.generated.GeneratedCollector
 import org.apache.flink.table.types.logical.LogicalType
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CorrelateCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CorrelateCodeGenerator.scala
index 89f0a4c..7470b11 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CorrelateCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/CorrelateCodeGenerator.scala
@@ -29,17 +29,18 @@ import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow, JoinedRow}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getEvalMethodSignature
 import org.apache.flink.table.functions.utils.{TableSqlFunction, UserDefinedFunctionUtils}
-import org.apache.flink.table.generated.GeneratedCollector
+import org.apache.flink.table.runtime.generated.GeneratedCollector
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
 import org.apache.flink.table.plan.schema.FlinkTableFunction
 import org.apache.flink.table.plan.util.RelExplainUtil
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
 import org.apache.flink.table.runtime.collector.TableFunctionCollector
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
 import org.apache.flink.table.runtime.util.StreamRecordCollector
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
-import org.apache.flink.table.types.{DataType, PlannerTypeUtils}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.types.DataType
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.JoinRelType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/EqualiserCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/EqualiserCodeGenerator.scala
index 8a5d9ea..4e0cfcb 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/EqualiserCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/EqualiserCodeGenerator.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.codegen
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.Indenter.toISC
-import org.apache.flink.table.generated.{GeneratedRecordEqualiser, RecordEqualiser}
-import org.apache.flink.table.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.generated.{GeneratedRecordEqualiser, RecordEqualiser}
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpandCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpandCodeGenerator.scala
index 9b2cdba..68dd542 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpandCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpandCodeGenerator.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.codegen
 
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.dataformat.{BaseRow, BoxedWrapperRow}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.rex.RexNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExprCodeGenerator.scala
index d4ff105..b06a42a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExprCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExprCodeGenerator.scala
@@ -27,15 +27,16 @@ import org.apache.flink.table.calcite.{FlinkTypeFactory, RexAggLocalVariable, Re
 import org.apache.flink.table.codegen.CodeGenUtils.{requireTemporal, requireTimeInterval, _}
 import org.apache.flink.table.codegen.GenerateUtils._
 import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE}
-import org.apache.flink.table.codegen.calls.{StringCallGen, FunctionGenerator, ScalarFunctionCallGen, TableFunctionCallGen}
+import org.apache.flink.table.codegen.calls.{FunctionGenerator, ScalarFunctionCallGen, StringCallGen, TableFunctionCallGen}
 import org.apache.flink.table.codegen.calls.ScalarOperatorGens._
 import org.apache.flink.table.dataformat._
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable._
 import org.apache.flink.table.functions.utils.{ScalarSqlFunction, TableSqlFunction}
-import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.typeutils.TypeCheckUtils.{isNumeric, isTemporal, isTimeInterval}
-import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TypeCheckUtils}
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isNumeric, isTemporal, isTimeInterval}
+import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
 
 import scala.collection.JavaConversions._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala
index fc4d77e..f24b94f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala
@@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.functions.ProcessFunction
 import org.apache.flink.streaming.api.functions.async.{AsyncFunction, RichAsyncFunction}
 import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.Indenter.toISC
-import org.apache.flink.table.generated.{GeneratedFunction, GeneratedJoinCondition, JoinCondition}
+import org.apache.flink.table.runtime.generated.{GeneratedFunction, GeneratedJoinCondition, JoinCondition}
 import org.apache.flink.table.types.logical.LogicalType
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GenerateUtils.scala
index 5b4ad0f..6aa30d3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GenerateUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GenerateUtils.scala
@@ -27,10 +27,10 @@ import org.apache.flink.table.codegen.calls.CurrentTimePointCallGen
 import org.apache.flink.table.dataformat._
 import org.apache.flink.table.plan.util.SortUtil
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime
-import org.apache.flink.table.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal}
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal}
 
 import org.apache.calcite.avatica.util.ByteString
 import org.apache.commons.lang3.StringEscapeUtils
@@ -661,7 +661,8 @@ object GenerateUtils {
     case _ if PlannerTypeUtils.isPrimitive(t) =>
       s"($leftTerm > $rightTerm ? 1 : $leftTerm < $rightTerm ? -1 : 0)"
     case VARBINARY | BINARY =>
-      val sortUtil = classOf[org.apache.flink.table.runtime.sort.SortUtil].getCanonicalName
+      val sortUtil = classOf[org.apache.flink.table.runtime.operators.sort.SortUtil]
+        .getCanonicalName
       s"$sortUtil.compareBinary($leftTerm, $rightTerm)"
     case ARRAY =>
       val at = t.asInstanceOf[ArrayType]
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GeneratedExpression.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GeneratedExpression.scala
index cd5e5ca..86afcf8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GeneratedExpression.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/GeneratedExpression.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.table.codegen
 
 import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForType, newName}
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
 import org.apache.flink.table.types.logical.LogicalType
-import org.apache.flink.table.typeutils.TypeCheckUtils
 
 /**
   * Describes a generated expression.
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/HashCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/HashCodeGenerator.scala
index 88fdccf..e6b7857 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/HashCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/HashCodeGenerator.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.codegen
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, hashCodeForType, newName}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.{GeneratedHashFunction, HashFunction}
+import org.apache.flink.table.runtime.generated.{GeneratedHashFunction, HashFunction}
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.util.MathUtils
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala
index ecf750e..de599b4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.common.io.GenericInputFormat
 import org.apache.flink.table.codegen.CodeGenUtils.newName
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.dataformat.GenericRow
-import org.apache.flink.table.generated.GeneratedInput
+import org.apache.flink.table.runtime.generated.GeneratedInput
 import org.apache.flink.table.types.logical.LogicalType
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LongHashJoinGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LongHashJoinGenerator.scala
index e144b8d..aebc27d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LongHashJoinGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LongHashJoinGenerator.scala
@@ -22,17 +22,15 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.metrics.Gauge
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, BINARY_ROW, baseRowFieldReadAccess, className, newName}
-import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
-import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, BINARY_ROW, baseRowFieldReadAccess, newName}
 import org.apache.flink.table.codegen.OperatorCodeGenerator.{INPUT_SELECTION, generateCollect}
 import org.apache.flink.table.dataformat.{BaseRow, JoinedRow}
-import org.apache.flink.table.generated.{GeneratedJoinCondition, GeneratedProjection}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.generated.{GeneratedJoinCondition, GeneratedProjection}
 import org.apache.flink.table.runtime.hashtable.{LongHashPartition, LongHybridHashTable}
-import org.apache.flink.table.runtime.join.HashJoinType
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.join.HashJoinType
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.typeutils.BinaryRowSerializer
 
 /**
   * Generate a long key hash join operator using [[LongHybridHashTable]].
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LookupJoinCodeGenerator.scala
index eb9fff7..a40695a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LookupJoinCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/LookupJoinCodeGenerator.scala
@@ -30,11 +30,11 @@ import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.dataformat.DataFormatConverters.DataFormatConverter
 import org.apache.flink.table.dataformat.{BaseRow, DataFormatConverters, GenericRow, JoinedRow}
 import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction}
-import org.apache.flink.table.generated.{GeneratedCollector, GeneratedFunction, GeneratedResultFuture}
+import org.apache.flink.table.runtime.generated.{GeneratedCollector, GeneratedFunction, GeneratedResultFuture}
 import org.apache.flink.table.plan.util.LookupJoinUtil.{ConstantLookupKey, FieldRefLookupKey, LookupKey}
 import org.apache.flink.table.runtime.collector.{TableFunctionCollector, TableFunctionResultFuture}
-import org.apache.flink.table.runtime.join.lookup.DelegatingResultFuture
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.operators.join.lookup.DelegatingResultFuture
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
 import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.types.Row
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
index f16a1e7..f0a9836 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
@@ -36,12 +36,12 @@ import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen.MatchCodeGenerator._
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable._
-import org.apache.flink.table.generated.GeneratedFunction
+import org.apache.flink.table.runtime.generated.GeneratedFunction
 import org.apache.flink.table.plan.util.AggregateUtil
 import org.apache.flink.table.plan.util.MatchUtil.AggregationPatternVariableFinder
-import org.apache.flink.table.runtime.`match`.{IterativeConditionRunner, PatternProcessFunctionRunner}
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore
+import org.apache.flink.table.runtime.operators.`match`.{IterativeConditionRunner, PatternProcessFunctionRunner}
 import org.apache.flink.table.types.logical.{RowType, TimestampKind, TimestampType}
 import org.apache.flink.table.utils.EncodingUtils
 import org.apache.flink.util.Collector
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
index aa42bee..5785e2c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
@@ -23,11 +23,11 @@ import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, BINARY_ROW, DEFAUL
 import org.apache.flink.table.codegen.OperatorCodeGenerator.{INPUT_SELECTION, generateCollect}
 import org.apache.flink.table.dataformat.{BaseRow, JoinedRow}
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.runtime.join.FlinkJoinType
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.AbstractRowSerializer
 
 import org.apache.calcite.rex.RexNode
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/OperatorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/OperatorCodeGenerator.scala
index b3e138d..37d4f63 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/OperatorCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/OperatorCodeGenerator.scala
@@ -24,7 +24,7 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.Indenter.toISC
-import org.apache.flink.table.generated.GeneratedOperator
+import org.apache.flink.table.runtime.generated.GeneratedOperator
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.util.Logging
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ProjectionCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ProjectionCodeGenerator.scala
index 82c918f..4f306b0 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ProjectionCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ProjectionCodeGenerator.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.codegen.CodeGenUtils._
 import org.apache.flink.table.codegen.GenerateUtils.generateRecordStatement
 import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE}
 import org.apache.flink.table.dataformat._
-import org.apache.flink.table.generated.{GeneratedProjection, Projection}
+import org.apache.flink.table.runtime.generated.{GeneratedProjection, Projection}
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
 
 import scala.collection.mutable
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/SinkCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/SinkCodeGenerator.scala
index 2559a0a..832e7ef 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/SinkCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/SinkCodeGenerator.scala
@@ -31,11 +31,12 @@ import org.apache.flink.table.codegen.CodeGenUtils.genToExternal
 import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
 import org.apache.flink.table.dataformat.util.BaseRowUtil
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
 import org.apache.flink.table.sinks.{DataStreamTableSink, TableSink}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
-import org.apache.flink.table.typeutils.{BaseRowTypeInfo, TimeIndicatorTypeInfo}
+import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
 import org.apache.flink.types.Row
 
 object SinkCodeGenerator {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ValuesCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ValuesCodeGenerator.scala
index 71956cb..db9b754 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ValuesCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ValuesCodeGenerator.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.codegen
 import org.apache.flink.table.api.{TableConfig, TableEnvironment}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
-import org.apache.flink.table.runtime.values.ValuesInputFormat
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.values.ValuesInputFormat
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex.RexLiteral
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGenerator.scala
index bf40b04..d2f609c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGenerator.scala
@@ -22,18 +22,17 @@ import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, _}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen._
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator._
-import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
+import org.apache.flink.table.dataformat.GenericRow
 import org.apache.flink.table.dataview._
 import org.apache.flink.table.expressions._
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
-import org.apache.flink.table.generated.{AggsHandleFunction, GeneratedAggsHandleFunction, GeneratedNamespaceAggsHandleFunction, NamespaceAggsHandleFunction}
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.{DataType, LogicalTypeDataTypeConverter}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.dataview.{StateListView, StateMapView}
+import org.apache.flink.table.runtime.generated.{AggsHandleFunction, GeneratedAggsHandleFunction, GeneratedNamespaceAggsHandleFunction, NamespaceAggsHandleFunction}
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.{BooleanType, IntType, LogicalType, RowType}
-import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 
 import org.apache.calcite.rex.RexLiteral
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DeclarativeAggCodeGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DeclarativeAggCodeGen.scala
index a3bf6e0..8db9ae8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DeclarativeAggCodeGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DeclarativeAggCodeGen.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator,
 import org.apache.flink.table.expressions.{ResolvedDistinctKeyReference, _}
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
 import org.apache.flink.table.plan.util.AggregateInfo
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.calcite.tools.RelBuilder
 import org.apache.flink.table.expressions.utils.ApiExpressionUtils
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DistinctAggCodeGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DistinctAggCodeGen.scala
index 2898bc6..79c35d3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DistinctAggCodeGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/DistinctAggCodeGen.scala
@@ -29,7 +29,7 @@ import org.apache.flink.table.dataformat.GenericRow
 import org.apache.flink.table.expressions.{Expression, RexNodeConverter}
 import org.apache.flink.table.plan.util.DistinctInfo
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
 import org.apache.flink.util.Preconditions
 import org.apache.flink.util.Preconditions.checkArgument
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/ImperativeAggCodeGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/ImperativeAggCodeGen.scala
index 5d0c7ee..b1106f5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/ImperativeAggCodeGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/ImperativeAggCodeGen.scala
@@ -27,10 +27,11 @@ import org.apache.flink.table.expressions.{Expression, ResolvedAggInputReference
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getAggFunctionUDIMethod, getAggUserDefinedInputTypes, getUserDefinedMethod, internalTypesToClasses, signatureToString}
 import org.apache.flink.table.plan.util.AggregateInfo
-import org.apache.flink.table.types.ClassLogicalTypeConverter.getInternalClassForType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getInternalClassForType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.{ClassDataTypeConverter, PlannerTypeUtils}
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
-import org.apache.flink.table.types.{ClassDataTypeConverter, DataType, PlannerTypeUtils}
 import org.apache.flink.table.util.SingleElementIterator
 
 import org.apache.calcite.tools.RelBuilder
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggCodeGenHelper.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggCodeGenHelper.scala
index 8e7d31de..df1ac42 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggCodeGenHelper.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggCodeGenHelper.scala
@@ -18,9 +18,6 @@
 
 package org.apache.flink.table.codegen.agg.batch
 
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
 import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.runtime.util.SingleElementIterator
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
@@ -33,12 +30,17 @@ import org.apache.flink.table.expressions.{Expression, ExpressionVisitor, FieldR
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getAccumulatorTypeOfAggregateFunction, getAggUserDefinedInputTypes}
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
-import org.apache.flink.table.generated.{GeneratedAggsHandleFunction, GeneratedOperator}
 import org.apache.flink.table.runtime.context.ExecutionContextImpl
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.generated.{GeneratedAggsHandleFunction, GeneratedOperator}
+import org.apache.flink.table.runtime.types.InternalSerializers
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
-import org.apache.flink.table.types.{DataType, InternalSerializers}
+
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
 
 import scala.collection.JavaConverters._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala
index f4bd20f..196786f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysCodeGenerator.scala
@@ -19,14 +19,14 @@
 package org.apache.flink.table.codegen.agg.batch
 
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
-import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext}
 import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
 import org.apache.flink.table.codegen.agg.batch.AggCodeGenHelper.genSortAggCodes
+import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.functions.AggregateFunction
-import org.apache.flink.table.generated.GeneratedOperator
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.runtime.TableStreamOperator
+import org.apache.flink.table.runtime.generated.GeneratedOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.tools.RelBuilder
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenHelper.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenHelper.scala
index e2e9f83..fa7d363 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenHelper.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenHelper.scala
@@ -31,13 +31,13 @@ import org.apache.flink.table.expressions.utils.ApiExpressionUtils
 import org.apache.flink.table.expressions.{Expression, ExpressionVisitor, FieldReferenceExpression, ResolvedAggInputReference, RexNodeConverter, TypeLiteralExpression, UnresolvedCallExpression, UnresolvedReferenceExpression, ValueLiteralExpression, _}
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
-import org.apache.flink.table.generated.{NormalizedKeyComputer, RecordComparator}
+import org.apache.flink.table.runtime.generated.{NormalizedKeyComputer, RecordComparator}
 import org.apache.flink.table.plan.util.SortUtil
-import org.apache.flink.table.runtime.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
-import org.apache.flink.table.runtime.sort.BufferedKVExternalSorter
+import org.apache.flink.table.runtime.operators.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
+import org.apache.flink.table.runtime.operators.sort.BufferedKVExternalSorter
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer
 import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
-import org.apache.flink.table.typeutils.BinaryRowSerializer
 
 import scala.collection.JavaConversions._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenerator.scala
index c374be7..52bbe8e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGenerator.scala
@@ -23,10 +23,10 @@ import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, Proje
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, GenericRow, JoinedRow}
 import org.apache.flink.table.functions.UserDefinedFunction
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
-import org.apache.flink.table.generated.GeneratedOperator
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.runtime.TableStreamOperator
-import org.apache.flink.table.runtime.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
+import org.apache.flink.table.runtime.generated.GeneratedOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
+import org.apache.flink.table.runtime.operators.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.tools.RelBuilder
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashWindowCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashWindowCodeGenerator.scala
index 21f7200..bea1283 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashWindowCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/HashWindowCodeGenerator.scala
@@ -23,7 +23,6 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo
 import org.apache.flink.runtime.operators.sort.QuickSort
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
 import org.apache.flink.table.api.Types
-import org.apache.flink.table.api.window.TimeWindow
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.codegen.CodeGenUtils.{BINARY_ROW, newName}
 import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
@@ -31,15 +30,16 @@ import org.apache.flink.table.codegen.agg.batch.AggCodeGenHelper.genGroupKeyChan
 import org.apache.flink.table.codegen.agg.batch.HashAggCodeGenHelper.{genHashAggOutputExpr, genRetryAppendToMap, prepareHashAggKVTypes, prepareHashAggMap}
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ExprCodeGenerator, GenerateUtils, GeneratedExpression, ProjectionCodeGenerator}
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow}
-import org.apache.flink.table.generated.GeneratedOperator
 import org.apache.flink.table.plan.logical.{LogicalWindow, SlidingGroupWindow, TumblingGroupWindow}
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.runtime.TableStreamOperator
-import org.apache.flink.table.runtime.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
-import org.apache.flink.table.runtime.sort.BinaryKVInMemorySortBuffer
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.generated.GeneratedOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
+import org.apache.flink.table.runtime.operators.aggregate.{BytesHashMap, BytesHashMapSpillMemorySegmentPool}
+import org.apache.flink.table.runtime.operators.sort.BinaryKVInMemorySortBuffer
+import org.apache.flink.table.runtime.operators.window.TimeWindow
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
-import org.apache.flink.table.typeutils.BinaryRowSerializer
 import org.apache.flink.util.MutableObjectIterator
 
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGenerator.scala
index 7a6a9f0..ed852cc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGenerator.scala
@@ -23,9 +23,9 @@ import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ProjectionCodeGenerator}
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, JoinedRow}
 import org.apache.flink.table.functions.AggregateFunction
-import org.apache.flink.table.generated.GeneratedOperator
+import org.apache.flink.table.runtime.generated.GeneratedOperator
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.runtime.TableStreamOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.tools.RelBuilder
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortWindowCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortWindowCodeGenerator.scala
index c97a2ef..490a9e8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortWindowCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/SortWindowCodeGenerator.scala
@@ -19,17 +19,17 @@
 package org.apache.flink.table.codegen.agg.batch
 
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
-import org.apache.flink.table.api.window.TimeWindow
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.codegen.CodeGenUtils.BINARY_ROW
 import org.apache.flink.table.codegen.agg.batch.AggCodeGenHelper.genGroupKeyChangedCheckCode
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ProjectionCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.functions.AggregateFunction
-import org.apache.flink.table.generated.GeneratedOperator
+import org.apache.flink.table.runtime.generated.GeneratedOperator
 import org.apache.flink.table.plan.logical.{LogicalWindow, SlidingGroupWindow, TumblingGroupWindow}
 import org.apache.flink.table.plan.util.AggregateInfoList
-import org.apache.flink.table.runtime.TableStreamOperator
+import org.apache.flink.table.runtime.operators.TableStreamOperator
+import org.apache.flink.table.runtime.operators.window.TimeWindow
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/WindowCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/WindowCodeGenerator.scala
index 060036f..6f6feb5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/WindowCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/agg/batch/WindowCodeGenerator.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.codegen.agg.batch
 
 import org.apache.flink.table.JLong
 import org.apache.flink.table.api.DataTypes
-import org.apache.flink.table.api.window.TimeWindow
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenUtils.{BINARY_ROW, boxedTypeTermForType, newName}
@@ -39,9 +38,10 @@ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getAccumu
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
 import org.apache.flink.table.plan.logical.{LogicalWindow, SlidingGroupWindow, TumblingGroupWindow}
 import org.apache.flink.table.plan.util.{AggregateInfoList, AggregateUtil}
+import org.apache.flink.table.runtime.operators.window.TimeWindow
 import org.apache.flink.table.runtime.util.RowIterator
-import org.apache.flink.table.runtime.window.grouping.{HeapWindowsGrouping, WindowsGrouping}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.operators.window.grouping.{HeapWindowsGrouping, WindowsGrouping}
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.LogicalTypeRoot.INTERVAL_DAY_TIME
 import org.apache.flink.table.types.logical._
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/DivCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/DivCallGen.scala
index 5b95f5c..b96f67c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/DivCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/DivCallGen.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.codegen.CodeGenUtils.DECIMAL_TERM
 import org.apache.flink.table.codegen.GenerateUtils.generateCallIfArgsNotNull
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, GeneratedExpression}
 import org.apache.flink.table.types.logical.{DecimalType, LogicalType}
-import org.apache.flink.table.typeutils.TypeCheckUtils.isNumeric
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isNumeric
 
 // DIV(T1, T2) - return integral part of the division; fractional truncated.
 //   T1, T2 are exact numeric types.
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
index 112dbcc..c61b420 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.codegen.calls
 
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable._
-import org.apache.flink.table.types.PlannerTypeUtils.isPrimitive
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isPrimitive
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/PrintCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/PrintCallGen.scala
index c6491a8..6f922aa 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/PrintCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/PrintCallGen.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.codegen.calls
 import org.apache.flink.table.codegen.CodeGenUtils.{newNames, primitiveTypeTermForType}
 import org.apache.flink.table.codegen.{CodeGeneratorContext, GeneratedExpression}
 import org.apache.flink.table.types.logical.LogicalType
-import org.apache.flink.table.typeutils.TypeCheckUtils.isBinaryString
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isBinaryString
 
 /**
   * Generates PRINT function call.
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
index 26c97d1..ac3217c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.dataformat.DataFormatConverters.getConverterForDat
 import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
index af713b4..93ec2c1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperatorGens.scala
@@ -23,13 +23,14 @@ import org.apache.flink.table.codegen.GenerateUtils._
 import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE}
 import org.apache.flink.table.codegen.{CodeGenException, CodeGeneratorContext, GeneratedExpression}
 import org.apache.flink.table.dataformat._
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
-import org.apache.flink.table.types.PlannerTypeUtils
-import org.apache.flink.table.types.PlannerTypeUtils.{isInteroperable, isPrimitive}
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.{isInteroperable, isPrimitive}
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils._
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.typeutils.TypeCheckUtils._
-import org.apache.flink.table.typeutils.{TypeCheckUtils, TypeCoercion}
+import org.apache.flink.table.typeutils.TypeCoercion
 import org.apache.flink.util.Preconditions.checkArgument
 
 import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
index aae7e87..96d1a6b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/StringCallGen.scala
@@ -27,7 +27,7 @@ import org.apache.flink.table.dataformat.DataFormatConverters
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable._
 import org.apache.flink.table.runtime.functions.SqlFunctionUtils
 import org.apache.flink.table.types.logical.{BooleanType, IntType, LogicalType, MapType, VarBinaryType, VarCharType}
-import org.apache.flink.table.typeutils.TypeCheckUtils.{isCharacterString, isTimestamp, isTimestampWithLocalZone}
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isTimestamp, isTimestampWithLocalZone}
 
 import org.apache.calcite.runtime.SqlFunctions
 import org.apache.calcite.sql.SqlOperator
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
index e879461..d3c065e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.codegen.GeneratedExpression.NEVER_NULL
 import org.apache.flink.table.codegen.{CodeGeneratorContext, GeneratedExpression}
 import org.apache.flink.table.functions.TableFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.getEvalMethodSignature
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.types.utils.TypeConversions
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala
index c411bb2..0ac8081 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/MultiFieldRangeBoundComparatorCodeGenerator.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, newName}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, GenerateUtils}
-import org.apache.flink.table.generated.{GeneratedRecordComparator, RecordComparator}
+import org.apache.flink.table.runtime.generated.{GeneratedRecordComparator, RecordComparator}
 import org.apache.flink.table.types.logical.{LogicalType, RowType}
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/RangeBoundComparatorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/RangeBoundComparatorCodeGenerator.scala
index e77b9da..9056afd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/RangeBoundComparatorCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/over/RangeBoundComparatorCodeGenerator.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, newName}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ExprCodeGenerator, GenerateUtils}
-import org.apache.flink.table.generated.{GeneratedRecordComparator, RecordComparator}
+import org.apache.flink.table.runtime.generated.{GeneratedRecordComparator, RecordComparator}
 import org.apache.flink.table.types.logical.{BigIntType, IntType, LogicalType, LogicalTypeRoot, RowType}
 
 import org.apache.calcite.avatica.util.DateTimeUtils
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/ComparatorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/ComparatorCodeGenerator.scala
index d929e36..1ef0179 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/ComparatorCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/ComparatorCodeGenerator.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, newName}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.codegen.{CodeGeneratorContext, GenerateUtils}
-import org.apache.flink.table.generated.{GeneratedRecordComparator, RecordComparator}
+import org.apache.flink.table.runtime.generated.{GeneratedRecordComparator, RecordComparator}
 import org.apache.flink.table.types.logical.LogicalType
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/SortCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/SortCodeGenerator.scala
index 99c52f1..59bea4c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/SortCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/sort/SortCodeGenerator.scala
@@ -22,9 +22,9 @@ import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, SEGMENT, newName}
 import org.apache.flink.table.codegen.Indenter.toISC
 import org.apache.flink.table.dataformat.{BinaryRow, Decimal}
-import org.apache.flink.table.generated.{GeneratedNormalizedKeyComputer, GeneratedRecordComparator, NormalizedKeyComputer, RecordComparator}
-import org.apache.flink.table.runtime.sort.SortUtil
-import org.apache.flink.table.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.generated.{GeneratedNormalizedKeyComputer, GeneratedRecordComparator, NormalizedKeyComputer, RecordComparator}
+import org.apache.flink.table.runtime.operators.sort.SortUtil
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
 import org.apache.flink.table.types.logical.{DecimalType, LogicalType}
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewUtils.scala
index fe8d43a..d31a340 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/dataview/DataViewUtils.scala
@@ -25,7 +25,8 @@ import org.apache.flink.table.api.dataview._
 import org.apache.flink.table.dataformat.{BinaryGeneric, GenericRow}
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.LegacyTypeInformationType
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 import org.apache.flink.table.typeutils._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala
index f53aa1e..359e54c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/PlannerExpressionConverter.scala
@@ -20,12 +20,12 @@ package org.apache.flink.table.expressions
 
 import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
 import org.apache.flink.table.api.{TableException, ValidationException}
-import org.apache.flink.table.functions.BuiltInFunctionDefinitions._
 import org.apache.flink.table.expressions.{E => PlannerE, UUID => PlannerUUID}
+import org.apache.flink.table.functions.BuiltInFunctionDefinitions._
 import org.apache.flink.table.functions._
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot.{CHAR, DECIMAL, SYMBOL, TIMESTAMP_WITHOUT_TIME_ZONE}
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks._
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 
 import _root_.scala.collection.JavaConverters._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/Reinterpret.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/Reinterpret.scala
index 530fd3c..d48ec4d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/Reinterpret.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/Reinterpret.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.expressions
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.typeutils.TypeCoercion
 import org.apache.flink.table.validate._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ReturnTypeInference.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ReturnTypeInference.scala
index 2a333ad..26477c0 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ReturnTypeInference.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/ReturnTypeInference.scala
@@ -20,9 +20,10 @@ package org.apache.flink.table.expressions
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkTypeSystem}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
+import org.apache.flink.table.runtime.typeutils.{BigDecimalTypeInfo, DecimalTypeInfo}
 import org.apache.flink.table.types.logical.{DecimalType, LogicalType}
-import org.apache.flink.table.typeutils.{BigDecimalTypeInfo, DecimalTypeInfo, TypeCoercion}
+import org.apache.flink.table.typeutils.TypeCoercion
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql.`type`.SqlTypeUtil
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/aggregations.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
index 27dadfc..be26470 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.java.typeutils.MultisetTypeInfo
 import org.apache.flink.table.calcite.FlinkTypeSystem
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFunction, UserDefinedAggregateFunction}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 import org.apache.flink.table.typeutils.TypeInfoCheckUtils
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
index 20a4ba2..2d3a2b3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.expressions
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.{fromLogicalTypeToTypeInfo, fromTypeInfoToLogicalType}
 import org.apache.flink.table.typeutils.TypeCoercion
 import org.apache.flink.table.typeutils.TypeInfoCheckUtils._
 import org.apache.flink.table.validate._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/call.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/call.scala
index 46059b1..95a2834 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/call.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/call.scala
@@ -20,9 +20,9 @@ package org.apache.flink.table.expressions
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.table.functions._
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/cast.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/cast.scala
index f546501..0ac5733 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/cast.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/cast.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.table.expressions
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.typeutils.TypeCoercion
 import org.apache.flink.table.validate._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
index 5da7e49..a43f442 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.calcite.FlinkTypeFactory._
 import org.apache.flink.table.operations.QueryOperation
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
 import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala
index 2012af0..570f192 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala
@@ -23,8 +23,8 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.functions.utils.AggSqlFunction.{createOperandTypeChecker, createOperandTypeInference, createReturnTypeInference}
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.LogicalType
 
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
index e873241..ea12c4b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
@@ -23,9 +23,9 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.functions.utils.ScalarSqlFunction._
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getOperandType, _}
-import org.apache.flink.table.types.ClassLogicalTypeConverter.getDefaultExternalClassForType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getDefaultExternalClassForType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.sql._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
index 966ab51..055c7ae 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
@@ -25,7 +25,7 @@ import org.apache.flink.table.functions.utils.TableSqlFunction._
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.plan.schema.FlinkTableFunction
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
 import org.apache.calcite.sql._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
index 7630fe6..e77c5c5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
@@ -27,16 +27,16 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.{BaseRow, BinaryString, Decimal}
 import org.apache.flink.table.functions._
 import org.apache.flink.table.plan.schema.DeferredTypeFlinkTableFunction
-import org.apache.flink.table.types.ClassDataTypeConverter.fromClassToDataType
-import org.apache.flink.table.types.ClassLogicalTypeConverter.{getDefaultExternalClassForType, getInternalClassForType}
+import org.apache.flink.table.runtime.types.ClassDataTypeConverter.fromClassToDataType
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.{getDefaultExternalClassForType, getInternalClassForType}
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType}
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType}
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.{LogicalType, LogicalTypeRoot, RowType}
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 import org.apache.flink.table.typeutils.FieldInfoUtils
-import org.apache.flink.table.typeutils.TypeCheckUtils.isAny
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isAny
 import org.apache.flink.types.Row
 import org.apache.flink.util.InstantiationUtil
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnInterval.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnInterval.scala
index ed13607..2b7f955 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnInterval.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnInterval.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.plan.nodes.physical.stream._
 import org.apache.flink.table.plan.schema.FlinkRelOptTable
 import org.apache.flink.table.plan.stats._
 import org.apache.flink.table.plan.util.{AggregateUtil, ColumnIntervalUtil, FlinkRelOptUtil, RankUtil}
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, VariableRankRange}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, VariableRankRange}
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.plan.volcano.RelSubset
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniqueness.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniqueness.scala
index 0e5dcd6..4f1fb61 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniqueness.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniqueness.scala
@@ -29,7 +29,7 @@ import org.apache.flink.table.plan.nodes.physical.batch._
 import org.apache.flink.table.plan.nodes.physical.stream._
 import org.apache.flink.table.plan.schema.FlinkRelOptTable
 import org.apache.flink.table.plan.util.{FlinkRelMdUtil, RankUtil}
-import org.apache.flink.table.runtime.rank.RankType
+import org.apache.flink.table.runtime.operators.rank.RankType
 import org.apache.flink.table.sources.TableSource
 
 import org.apache.calcite.plan.RelOptTable
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeys.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeys.scala
index 89ba591..09fa93c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeys.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdUniqueKeys.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.plan.nodes.physical.batch._
 import org.apache.flink.table.plan.nodes.physical.stream._
 import org.apache.flink.table.plan.schema.FlinkRelOptTable
 import org.apache.flink.table.plan.util.{FlinkRelMdUtil, RankUtil}
-import org.apache.flink.table.runtime.rank.RankType
+import org.apache.flink.table.runtime.operators.rank.RankType
 import org.apache.flink.table.sources.TableSource
 import org.apache.flink.table.{JArrayList, JBoolean, JHashMap, JHashSet, JList, JSet}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalRank.scala
index e9e4c02..8c5871c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/LogicalRank.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.nodes.calcite
 
-import org.apache.flink.table.runtime.rank.{RankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType}
 
 import org.apache.calcite.plan.{Convention, RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataTypeField
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Rank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Rank.scala
index 2cec404..856cf5a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Rank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Rank.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.calcite
 
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.plan.util._
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankRange, RankType, VariableRankRange}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange, RankType, VariableRankRange}
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Sink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Sink.scala
index c58d46d..9e9baf1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Sink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/calcite/Sink.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.calcite
 
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.sinks.TableSink
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
index bde6b9c..2157803 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
@@ -36,15 +36,15 @@ import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction, User
 import org.apache.flink.table.plan.nodes.FlinkRelNode
 import org.apache.flink.table.plan.util.LookupJoinUtil._
 import org.apache.flink.table.plan.util.{JoinTypeUtil, RelExplainUtil}
-import org.apache.flink.table.runtime.join.lookup.{AsyncLookupJoinRunner, AsyncLookupJoinWithCalcRunner, LookupJoinRunner, LookupJoinWithCalcRunner}
+import org.apache.flink.table.runtime.operators.join.lookup.{AsyncLookupJoinRunner, AsyncLookupJoinWithCalcRunner, LookupJoinRunner, LookupJoinWithCalcRunner}
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getInternalClassForType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.sources.{LookupableTableSource, TableSource}
-import org.apache.flink.table.types.ClassLogicalTypeConverter
-import org.apache.flink.table.types.ClassLogicalTypeConverter.getInternalClassForType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
 import org.apache.flink.table.types.logical.{LogicalType, RowType, TypeInformationAnyType}
 import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalJoin.scala
index 4a0c303..02617fd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonPhysicalJoin.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.common
 
 import org.apache.flink.table.plan.nodes.physical.FlinkPhysicalRel
 import org.apache.flink.table.plan.util.{JoinTypeUtil, JoinUtil, RelExplainUtil}
-import org.apache.flink.table.runtime.join.FlinkJoinType
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRank.scala
index 8cd995e..a91e0f6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalRank.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.logical
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.calcite.{LogicalRank, Rank}
 import org.apache.flink.table.plan.util.RelExplainUtil
-import org.apache.flink.table.runtime.rank.{RankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType}
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.`type`.RelDataTypeField
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala
index a3f13f7..0ef5182 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala
@@ -29,7 +29,7 @@ import org.apache.flink.table.plan.nodes.common.CommonCalc
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.RelExplainUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
index 75c471e..2f3bfb7 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
@@ -31,9 +31,9 @@ import org.apache.flink.table.plan.nodes.common.CommonPhysicalExchange
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.BinaryHashPartitioner
+import org.apache.flink.table.runtime.partitioner.BinaryHashPartitioner
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.{RelDistribution, RelNode, RelWriter}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExpand.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExpand.scala
index 8e29d8a..2810eab 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExpand.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExpand.scala
@@ -27,7 +27,7 @@ import org.apache.flink.table.plan.nodes.calcite.Expand
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.RelExplainUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
index 0fa005e..681307c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
@@ -32,8 +32,8 @@ import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateInfoList
 import org.apache.flink.table.plan.util.FlinkRelMdUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
index d6af5c6..e0cd7d5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
@@ -33,9 +33,9 @@ import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.nodes.{ExpressionFormat, FlinkConventions}
 import org.apache.flink.table.plan.util.{FlinkRelMdUtil, JoinUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.join.{HashJoinOperator, HashJoinType}
+import org.apache.flink.table.runtime.operators.join.{HashJoinOperator, HashJoinType}
+import org.apache.flink.table.runtime.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.core._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
index b076e46..7e2b617 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
@@ -34,9 +34,9 @@ import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateInfoList
 import org.apache.flink.table.plan.util.FlinkRelMdUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.runtime.aggregate.BytesHashMap
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.aggregate.BytesHashMap
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala
index 8fac0d0..8439e53 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala
@@ -20,10 +20,10 @@ package org.apache.flink.table.plan.nodes.physical.batch
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.GeneratedJoinCondition
 import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef}
 import org.apache.flink.table.plan.nodes.common.CommonPhysicalJoin
 import org.apache.flink.table.plan.nodes.exec.BatchExecNode
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition
 import org.apache.flink.table.types.logical.RowType
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLimit.scala
index 52bae84..522c891 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLimit.scala
@@ -27,7 +27,7 @@ import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.RelExplainUtil.fetchToString
 import org.apache.flink.table.plan.util.SortUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.sort.LimitOperator
+import org.apache.flink.table.runtime.operators.sort.LimitOperator
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
index 62132d2..60c7924 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
@@ -30,7 +30,7 @@ import org.apache.flink.table.plan.nodes.ExpressionFormat
 import org.apache.flink.table.plan.nodes.exec.ExecNode
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
+import org.apache.flink.table.runtime.typeutils.{BaseRowTypeInfo, BinaryRowSerializer}
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.core._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
index 0331186..3b4b55b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
@@ -31,7 +31,6 @@ import org.apache.flink.table.codegen.over.{MultiFieldRangeBoundComparatorCodeGe
 import org.apache.flink.table.codegen.sort.ComparatorCodeGenerator
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow}
 import org.apache.flink.table.functions.UserDefinedFunction
-import org.apache.flink.table.generated.GeneratedRecordComparator
 import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef}
 import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory}
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
@@ -42,11 +41,12 @@ import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateI
 import org.apache.flink.table.plan.util.OverAggregateUtil.getLongBoundary
 import org.apache.flink.table.plan.util.{FlinkRelOptUtil, OverAggregateUtil, RelExplainUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.over.frame.OffsetOverFrame.CalcOffsetFunc
-import org.apache.flink.table.runtime.over.frame.{InsensitiveOverFrame, OffsetOverFrame, OverWindowFrame, RangeSlidingOverFrame, RangeUnboundedFollowingOverFrame, RangeUnboundedPrecedingOverFrame, RowSlidingOverFrame, RowUnboundedFollowingOverFrame, RowUnboundedPrecedingOverFrame, UnboundedOverWindowFrame}
-import org.apache.flink.table.runtime.over.{BufferDataOverWindowOperator, NonBufferOverWindowOperator}
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator
+import org.apache.flink.table.runtime.operators.over.frame.OffsetOverFrame.CalcOffsetFunc
+import org.apache.flink.table.runtime.operators.over.frame.{InsensitiveOverFrame, OffsetOverFrame, OverWindowFrame, RangeSlidingOverFrame, RangeUnboundedFollowingOverFrame, RangeUnboundedPrecedingOverFrame, RowSlidingOverFrame, RowUnboundedFollowingOverFrame, RowUnboundedPrecedingOverFrame, UnboundedOverWindowFrame}
+import org.apache.flink.table.runtime.operators.over.{BufferDataOverWindowOperator, NonBufferOverWindowOperator}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot.{BIGINT, INTEGER, SMALLINT}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelDistribution.Type._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
index 76b4c99..b9cf474 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
@@ -32,9 +32,9 @@ import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecJoinRuleBase
 import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankRange, RankType}
-import org.apache.flink.table.runtime.sort.RankOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange, RankType}
+import org.apache.flink.table.runtime.operators.sort.RankOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.RelDistribution.Type
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
index 9ce1d4c..b78f7e9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
@@ -30,10 +30,11 @@ import org.apache.flink.table.plan.nodes.calcite.Sink
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.planner.BatchPlanner
+import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.sinks.{DataStreamTableSink, RetractStreamTableSink, StreamTableSink, TableSink, UpsertStreamTableSink}
-import org.apache.flink.table.types.{ClassLogicalTypeConverter, DataType}
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
index 2babfb8..9cf1f60 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
@@ -30,8 +30,8 @@ import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.util.{FlinkRelMdUtil, RelExplainUtil, SortUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.sort.SortOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.sort.SortOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.core.Sort
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala
index 3e03f60..d93c35d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregateBase.scala
@@ -28,8 +28,8 @@ import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory}
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateInfoList
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortLimit.scala
index 3bf9916..4626774 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortLimit.scala
@@ -27,8 +27,8 @@ import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory}
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.{RelExplainUtil, SortUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.sort.SortLimitOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.sort.SortLimitOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.core.Sort
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
index 1808ba6..046f77b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
@@ -33,9 +33,9 @@ import org.apache.flink.table.plan.nodes.exec.ExecNode
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.util.{FlinkRelMdUtil, FlinkRelOptUtil, JoinUtil, SortUtil}
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.join.{FlinkJoinType, SortMergeJoinOperator}
+import org.apache.flink.table.runtime.operators.join.{FlinkJoinType, SortMergeJoinOperator}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.core._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
index c0157c9..3b02397 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
@@ -32,8 +32,8 @@ import org.apache.flink.table.plan.logical.LogicalWindow
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
 import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateInfoList
 import org.apache.flink.table.planner.BatchPlanner
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCalc.scala
index f40007d..0745d92 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCalc.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCalc.scala
@@ -27,8 +27,8 @@ import org.apache.flink.table.plan.nodes.common.CommonCalc
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.RelExplainUtil
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.AbstractProcessStreamOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCorrelate.scala
index d3324ac..dd41ca8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCorrelate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecCorrelate.scala
@@ -25,7 +25,7 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan
 import org.apache.flink.table.plan.util.RelExplainUtil
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.AbstractProcessStreamOperator
+import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDataStreamScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDataStreamScan.scala
index b2664ab..7ade5ad 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDataStreamScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDataStreamScan.scala
@@ -29,11 +29,11 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.schema.DataStreamTable
 import org.apache.flink.table.plan.util.ScanUtil
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.AbstractProcessStreamOperator
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils
 import org.apache.flink.table.types.logical.{RowType, TimestampKind, TimestampType}
 import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER
-import org.apache.flink.table.typeutils.TypeCheckUtils
 
 import org.apache.calcite.plan._
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
index 01cacf9..7103e60 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
@@ -28,9 +28,9 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util.{AggregateUtil, KeySelectorUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator
-import org.apache.flink.table.runtime.deduplicate.{DeduplicateKeepFirstRowFunction, DeduplicateKeepLastRowFunction, MiniBatchDeduplicateKeepFirstRowFunction, MiniBatchDeduplicateKeepLastRowFunction}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.operators.deduplicate.{DeduplicateKeepFirstRowFunction, DeduplicateKeepLastRowFunction, MiniBatchDeduplicateKeepFirstRowFunction, MiniBatchDeduplicateKeepLastRowFunction}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExchange.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExchange.scala
index 27cec1d..492b350 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExchange.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExchange.scala
@@ -28,7 +28,7 @@ import org.apache.flink.table.plan.nodes.common.CommonPhysicalExchange
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.KeySelectorUtil
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.{RelDistribution, RelNode}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExpand.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExpand.scala
index 23602be..1590d70 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExpand.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecExpand.scala
@@ -25,7 +25,7 @@ import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.nodes.calcite.Expand
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
index 1a9df7a..bc6ef23 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
@@ -25,17 +25,17 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.codegen.{CodeGeneratorContext, EqualiserCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction
 import org.apache.flink.table.plan.PartialFinalType
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util._
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.aggregate.MiniBatchGlobalGroupAggFunction
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.operators.aggregate.MiniBatchGlobalGroupAggFunction
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
index 30afbe3..d77fb09 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
@@ -31,10 +31,10 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util.{AggregateInfoList, AggregateUtil, RelExplainUtil, _}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.aggregate.{GroupAggFunction, MiniBatchGroupAggFunction}
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.aggregate.{GroupAggFunction, MiniBatchGroupAggFunction}
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
index 41ed54a..e2b48a4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
@@ -20,24 +20,23 @@ package org.apache.flink.table.plan.nodes.physical.stream
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.window.{CountWindow, TimeWindow}
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.codegen.{CodeGeneratorContext, EqualiserCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.{GeneratedNamespaceAggsHandleFunction, GeneratedRecordEqualiser}
 import org.apache.flink.table.plan.logical._
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util.AggregateUtil.{hasRowIntervalType, hasTimeIntervalType, isProctimeAttribute, isRowtimeAttribute, toDuration, toLong, transformToStreamAggregateInfoList}
 import org.apache.flink.table.plan.util.{AggregateInfoList, KeySelectorUtil, RelExplainUtil, WindowEmitStrategy}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.window.{WindowOperator, WindowOperatorBuilder}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.generated.{GeneratedNamespaceAggsHandleFunction, GeneratedRecordEqualiser}
+import org.apache.flink.table.runtime.operators.window.{CountWindow, TimeWindow, WindowOperator, WindowOperatorBuilder}
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.LogicalType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala
index de16837..0d4f291 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIncrementalGroupAggregate.scala
@@ -24,14 +24,14 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util._
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.aggregate.MiniBatchIncrementalGroupAggFunction
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.operators.aggregate.MiniBatchIncrementalGroupAggFunction
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecJoin.scala
index 1178e65..017f059 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecJoin.scala
@@ -26,10 +26,10 @@ import org.apache.flink.table.plan.nodes.common.CommonPhysicalJoin
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.{JoinUtil, KeySelectorUtil, RelExplainUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.join.FlinkJoinType
-import org.apache.flink.table.runtime.join.stream.state.JoinInputSideSpec
-import org.apache.flink.table.runtime.join.stream.{StreamingJoinOperator, StreamingSemiAntiJoinOperator}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec
+import org.apache.flink.table.runtime.operators.join.stream.{StreamingJoinOperator, StreamingSemiAntiJoinOperator}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan._
 import org.apache.calcite.plan.hep.HepRelVertex
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
index abd2e6d..2bd4785 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
@@ -31,8 +31,8 @@ import org.apache.flink.table.plan.util.RelExplainUtil._
 import org.apache.flink.table.plan.util.{RelExplainUtil, SortUtil}
 import org.apache.flink.table.planner.StreamPlanner
 import org.apache.flink.table.runtime.keyselector.NullBinaryRowKeySelector
-import org.apache.flink.table.runtime.rank.{AppendOnlyTopNFunction, ConstantRankRange, RankType, RetractableTopNFunction}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.rank.{AppendOnlyTopNFunction, ConstantRankRange, RankType, RetractableTopNFunction}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala
index 05d0c83..d10f36c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLocalGroupAggregate.scala
@@ -29,9 +29,9 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util._
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.aggregate.MiniBatchLocalGroupAggFunction
-import org.apache.flink.table.runtime.bundle.MapBundleOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.aggregate.MiniBatchLocalGroupAggFunction
+import org.apache.flink.table.runtime.operators.bundle.MapBundleOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecMatch.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecMatch.scala
index c73bb71..30ce8f5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecMatch.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecMatch.scala
@@ -41,9 +41,9 @@ import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRul
 import org.apache.flink.table.plan.util.RelExplainUtil._
 import org.apache.flink.table.plan.util.{KeySelectorUtil, RexDefaultVisitor, SortUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.`match`.{BaseRowEventComparator, RowtimeProcessFunction}
+import org.apache.flink.table.runtime.operators.`match`.{BaseRowEventComparator, RowtimeProcessFunction}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.util.MathUtils
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecOverAggregate.scala
index 54895ff..4a4ea19 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecOverAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecOverAggregate.scala
@@ -32,9 +32,9 @@ import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRul
 import org.apache.flink.table.plan.util.AggregateUtil.transformToStreamAggregateInfoList
 import org.apache.flink.table.plan.util.{KeySelectorUtil, OverAggregateUtil, RelExplainUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.over._
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.over._
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
 import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
index 7b404d0..cc5ddcc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
@@ -33,8 +33,8 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
 import org.apache.flink.table.plan.util._
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.rank._
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.rank._
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
index 8a01d69..2c479ec 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
@@ -32,10 +32,10 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil
 import org.apache.flink.table.plan.util.UpdatingPlanChecker
 import org.apache.flink.table.planner.StreamPlanner
+import org.apache.flink.table.runtime.typeutils.{BaseRowTypeInfo, TypeCheckUtils}
 import org.apache.flink.table.sinks._
 import org.apache.flink.table.types.logical.TimestampType
 import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
-import org.apache.flink.table.typeutils.{BaseRowTypeInfo, TypeCheckUtils}
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelNode
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
index e22e5b3..d71e414 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
@@ -30,8 +30,8 @@ import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.{RelExplainUtil, SortUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.sort.StreamSortOperator
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.sort.StreamSortOperator
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
index 6b5b7d5..481de91 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
@@ -30,8 +30,8 @@ import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRul
 import org.apache.flink.table.plan.util.{AppendFastStrategy, KeySelectorUtil, RankProcessStrategy, RelExplainUtil, RetractStrategy, SortUtil, UpdateFastStrategy}
 import org.apache.flink.table.planner.StreamPlanner
 import org.apache.flink.table.runtime.keyselector.NullBinaryRowKeySelector
-import org.apache.flink.table.runtime.rank.{AppendOnlyTopNFunction, ConstantRankRange, RankType, RetractableTopNFunction, UpdatableTopNFunction}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.rank.{AppendOnlyTopNFunction, ConstantRankRange, RankType, RetractableTopNFunction, UpdatableTopNFunction}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.core.Sort
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTableSourceScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
index a67f4c1..f45247f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTableSourceScan.scala
@@ -34,7 +34,7 @@ import org.apache.flink.table.plan.nodes.physical.PhysicalTableSourceScan
 import org.apache.flink.table.plan.schema.FlinkRelOptTable
 import org.apache.flink.table.plan.util.ScanUtil
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.AbstractProcessStreamOperator
+import org.apache.flink.table.runtime.operators.AbstractProcessStreamOperator
 import org.apache.flink.table.sources.wmstrategies.{PeriodicWatermarkAssigner, PreserveWatermarks, PunctuatedWatermarkAssigner}
 import org.apache.flink.table.sources.{RowtimeAttributeDescriptor, StreamTableSource, TableSourceUtil}
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalJoin.scala
index ac3f4de..8b569e8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalJoin.scala
@@ -27,16 +27,16 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.calcite.FlinkTypeFactory.{isProctimeIndicatorType, isRowtimeIndicatorType}
 import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.GeneratedJoinCondition
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition
 import org.apache.flink.table.plan.nodes.common.CommonPhysicalJoin
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.TemporalJoinUtil.TEMPORAL_JOIN_CONDITION
 import org.apache.flink.table.plan.util.{InputRefVisitor, KeySelectorUtil, RelExplainUtil, TemporalJoinUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.join.temporal.{TemporalProcessTimeJoinOperator, TemporalRowTimeJoinOperator}
+import org.apache.flink.table.runtime.operators.join.temporal.{TemporalProcessTimeJoinOperator, TemporalRowTimeJoinOperator}
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.util.Preconditions.checkState
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalSort.scala
index 1e755f4..7ff4001 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecTemporalSort.scala
@@ -28,8 +28,8 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.{RelExplainUtil, SortUtil}
 import org.apache.flink.table.planner.StreamPlanner
 import org.apache.flink.table.runtime.keyselector.NullBinaryRowKeySelector
-import org.apache.flink.table.runtime.sort.{ProcTimeSortOperator, RowTimeSortOperator}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.sort.{ProcTimeSortOperator, RowTimeSortOperator}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.RelFieldCollation.Direction
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
index dae7285..b4febb6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
@@ -28,8 +28,8 @@ import org.apache.flink.table.plan.`trait`.{MiniBatchIntervalTraitDef, MiniBatch
 import org.apache.flink.table.plan.nodes.calcite.WatermarkAssigner
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.watermarkassigner.{MiniBatchAssignerOperator, MiniBatchedWatermarkAssignerOperator, WatermarkAssignerOperator}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.wmassigners.{MiniBatchAssignerOperator, MiniBatchedWatermarkAssignerOperator, WatermarkAssignerOperator}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 import org.apache.flink.util.Preconditions
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWindowJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWindowJoin.scala
index 922cd03..e761b93 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWindowJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWindowJoin.scala
@@ -27,12 +27,12 @@ import org.apache.flink.streaming.api.transformations.{OneInputTransformation, T
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.generated.GeneratedFunction
+import org.apache.flink.table.runtime.generated.GeneratedFunction
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.util.{JoinTypeUtil, KeySelectorUtil, RelExplainUtil, UpdatingPlanChecker, WindowJoinUtil}
 import org.apache.flink.table.planner.StreamPlanner
-import org.apache.flink.table.runtime.join.{FlinkJoinType, KeyedCoProcessOperatorWithWatermarkDelay, OuterJoinPaddingUtil, ProcTimeBoundedStreamJoin, RowTimeBoundedStreamJoin}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.runtime.operators.join.{FlinkJoinType, KeyedCoProcessOperatorWithWatermarkDelay, OuterJoinPaddingUtil, ProcTimeBoundedStreamJoin, RowTimeBoundedStreamJoin}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.util.Collector
 
 import org.apache.calcite.plan._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/BatchLogicalWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/BatchLogicalWindowAggregateRule.scala
index 9ad3c96..499a0df 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/BatchLogicalWindowAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/BatchLogicalWindowAggregateRule.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalType
 import org.apache.flink.table.expressions.FieldReferenceExpression
 import org.apache.flink.table.plan.nodes.calcite.LogicalWindowAggregate
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRule.scala
index 39f7b89..3ed33ec 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/CalcRankTransposeRule.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.table.api.TableException
 import org.apache.flink.table.plan.nodes.logical._
 import org.apache.flink.table.plan.util.{FlinkRexUtil, RankUtil}
-import org.apache.flink.table.runtime.rank.VariableRankRange
+import org.apache.flink.table.runtime.operators.rank.VariableRankRange
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRule.scala
index 595ec18..2c9b245 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/FlinkLogicalRankRule.scala
@@ -21,7 +21,7 @@ import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalOverAggregate, FlinkLogicalRank}
 import org.apache.flink.table.plan.util.RankUtil
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, ConstantRankRangeWithoutEnd, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, ConstantRankRangeWithoutEnd, RankType}
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRule.scala
index 705dadd..f6f35b6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRule.scala
@@ -24,8 +24,8 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalType
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.plan.util.ExplodeFunctionUtil
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.logical.RowType
 
 import com.google.common.collect.ImmutableList
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalWindowAggregateRuleBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalWindowAggregateRuleBase.scala
index 905b3f3..8c7fffa 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalWindowAggregateRuleBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/LogicalWindowAggregateRuleBase.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.expressions.{FieldReferenceExpression, PlannerWind
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable
 import org.apache.flink.table.plan.logical.{LogicalWindow, SessionGroupWindow, SlidingGroupWindow, TumblingGroupWindow}
 import org.apache.flink.table.plan.nodes.calcite.LogicalWindowAggregate
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 
 import com.google.common.collect.ImmutableList
 import org.apache.calcite.plan.RelOptRule._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRule.scala
index 60ab14d0..6d9c2a4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/RankNumberColumnRemoveRule.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalRank}
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/StreamLogicalWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/StreamLogicalWindowAggregateRule.scala
index 53b1729..483056e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/StreamLogicalWindowAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/StreamLogicalWindowAggregateRule.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalType
 import org.apache.flink.table.expressions.FieldReferenceExpression
 import org.apache.flink.table.plan.nodes.calcite.LogicalWindowAggregate
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
index 430435b..63e070d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
@@ -25,7 +25,7 @@ import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunctio
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
 import org.apache.flink.table.plan.util.{AggregateUtil, FlinkRelOptUtil}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.util.AggregatePhaseStrategy
 import org.apache.flink.table.util.TableConfigUtils.getAggPhaseStrategy
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
index 9a24a16..a90a277 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalAggregate
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecHashAggregate, BatchExecLocalHashAggregate}
 import org.apache.flink.table.plan.util.{AggregateUtil, OperatorType}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLookupJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLookupJoinRule.scala
index b189094..5820e86 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLookupJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecLookupJoinRule.scala
@@ -23,8 +23,8 @@ import org.apache.flink.table.plan.nodes.common.CommonLookupJoin
 import org.apache.flink.table.plan.nodes.logical._
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecLookupJoin
 import org.apache.flink.table.plan.rules.physical.common.{BaseSnapshotOnCalcTableScanRule, BaseSnapshotOnTableScanRule}
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.sources.TableSource
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.{TimestampKind, TimestampType}
 
 import org.apache.calcite.plan.RelOptRule
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecRankRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecRankRule.scala
index f6ef77d..51e8210 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecRankRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecRankRule.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalRank
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecRank
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
 import org.apache.calcite.rel.convert.ConverterRule
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
index 6a177f2..a877d8e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalAggregate
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecLocalSortAggregate, BatchExecSortAggregate}
 import org.apache.flink.table.plan.util.{AggregateUtil, OperatorType}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
index c41e775..147ee15 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
@@ -30,7 +30,7 @@ import org.apache.flink.table.plan.nodes.logical.FlinkLogicalWindowAggregate
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecHashWindowAggregate, BatchExecLocalHashWindowAggregate, BatchExecLocalSortWindowAggregate, BatchExecSortWindowAggregate}
 import org.apache.flink.table.plan.util.AggregateUtil
 import org.apache.flink.table.plan.util.AggregateUtil.hasTimeIntervalType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.{BigIntType, IntType, LogicalType}
 
 import org.apache.calcite.plan.RelOptRule._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDeduplicateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDeduplicateRule.scala
index d983a13..9413561 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDeduplicateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecDeduplicateRule.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalRank
 import org.apache.flink.table.plan.nodes.physical.stream.{StreamExecDeduplicate, StreamExecRank}
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
 import org.apache.calcite.rel.`type`.RelDataType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLookupJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLookupJoinRule.scala
index 1c2ed95..c43694b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLookupJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecLookupJoinRule.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.plan.nodes.logical._
 import org.apache.flink.table.plan.nodes.physical.stream.StreamExecLookupJoin
 import org.apache.flink.table.plan.rules.physical.common.{BaseSnapshotOnCalcTableScanRule, BaseSnapshotOnTableScanRule}
 import org.apache.flink.table.sources.TableSource
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 
 import org.apache.calcite.plan.RelOptRule
 import org.apache.calcite.rex.RexProgram
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DeferredTypeFlinkTableFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DeferredTypeFlinkTableFunction.scala
index 036b01b..c64e52f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DeferredTypeFlinkTableFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/DeferredTypeFlinkTableFunction.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.schema
 import org.apache.flink.table.functions.TableFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.utils.TypeConversions
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/InlineTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/InlineTable.scala
index 41b1da9..0f6039b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/InlineTable.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/InlineTable.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.{LogicalType, RowType, TimestampKind, TimestampType}
 import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala
index 0708611..2b1ed23 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.schema
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.sinks.TableSink
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TypedFlinkTableFunction.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TypedFlinkTableFunction.scala
index acc1533..3c4b398 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TypedFlinkTableFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/TypedFlinkTableFunction.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.plan.schema
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.functions.TableFunction
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.typeutils.FieldInfoUtils
 
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggFunctionFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggFunctionFactory.scala
index 611d59c..2692063 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggFunctionFactory.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggFunctionFactory.scala
@@ -33,10 +33,10 @@ import org.apache.flink.table.functions.aggfunctions.SumWithRetractAggFunction._
 import org.apache.flink.table.functions.aggfunctions._
 import org.apache.flink.table.functions.sql.{SqlConcatAggFunction, SqlFirstLastValueAggFunction, SqlIncrSumAggFunction}
 import org.apache.flink.table.functions.utils.AggSqlFunction
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical._
-import org.apache.flink.table.typeutils.DecimalTypeInfo
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.AggregateCall
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
index d73fe33..69e1df8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
@@ -34,16 +34,16 @@ import org.apache.flink.table.functions.utils.AggSqlFunction
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
 import org.apache.flink.table.plan.`trait`.RelModifiedMonotonicity
-import org.apache.flink.table.runtime.bundle.trigger.CountBundleTrigger
+import org.apache.flink.table.runtime.operators.bundle.trigger.CountBundleTrigger
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType}
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.{fromDataTypeToLogicalType, fromLogicalTypeToDataType}
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot
 import org.apache.flink.table.types.logical.{LogicalTypeRoot, _}
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo
 
 import org.apache.calcite.rel.`type`._
 import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelMdUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelMdUtil.scala
index 49d67a8..1514fe8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelMdUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelMdUtil.scala
@@ -24,9 +24,9 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.BinaryRow
 import org.apache.flink.table.plan.nodes.calcite.{Expand, Rank, WindowAggregate}
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecGroupAggregateBase, BatchExecLocalHashWindowAggregate, BatchExecLocalSortWindowAggregate, BatchExecWindowAggregateBase}
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankRange}
-import org.apache.flink.table.runtime.sort.BinaryIndexedSortable
-import org.apache.flink.table.typeutils.BinaryRowSerializer
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange}
+import org.apache.flink.table.runtime.operators.sort.BinaryIndexedSortable
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer
 
 import com.google.common.collect.ImmutableList
 import org.apache.calcite.avatica.util.TimeUnitRange._
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/JoinUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/JoinUtil.scala
index 479be12..0ad79a8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/JoinUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/JoinUtil.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.util
 
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
-import org.apache.flink.table.generated.GeneratedJoinCondition
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.core.{Join, JoinInfo}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/PartitionPruner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/PartitionPruner.scala
index 532ade8..a68f475 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/PartitionPruner.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/PartitionPruner.scala
@@ -25,9 +25,9 @@ import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.codegen.CodeGenUtils.DEFAULT_COLLECTOR_TERM
 import org.apache.flink.table.codegen.{ConstantCodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
 import org.apache.flink.table.dataformat.{BinaryString, Decimal, GenericRow}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.{BooleanType, DecimalType, LogicalType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.rex.RexNode
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
index 0645666..f4b9fcd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.util
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.ExpressionReducer
 import org.apache.flink.table.plan.nodes.calcite.Rank
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, ConstantRankRangeWithoutEnd, RankRange, VariableRankRange}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, ConstantRankRangeWithoutEnd, RankRange, VariableRankRange}
 
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.rex.{RexBuilder, RexCall, RexInputRef, RexLiteral, RexNode, RexUtil}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeExtractor.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeExtractor.scala
index 89a6d62..9f4f09e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeExtractor.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RexNodeExtractor.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.expressions._
 import org.apache.flink.table.expressions.utils.ApiExpressionUtils._
 import org.apache.flink.table.functions.BuiltInFunctionDefinitions.{AND, CAST, OR}
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.table.util.Logging
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ScanUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ScanUtil.scala
index b17406b..359b3fa 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ScanUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/ScanUtil.scala
@@ -25,15 +25,17 @@ import org.apache.flink.table.codegen.CodeGenUtils.{DEFAULT_INPUT1_TERM, GENERIC
 import org.apache.flink.table.codegen.OperatorCodeGenerator.generateCollect
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ExprCodeGenerator, OperatorCodeGenerator}
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.{BaseRowTypeInfo, TimeIndicatorTypeInfo}
+import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
+
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.TableScan
 import org.apache.calcite.rex.RexNode
 import org.apache.flink.api.dag.Transformation
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 
 import scala.collection.JavaConversions._
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SetOpRewriteUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SetOpRewriteUtil.scala
index f300645..a28c867 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SetOpRewriteUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/SetOpRewriteUtil.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalRowType
 import org.apache.flink.table.functions.tablefunctions.ReplicateRows
 import org.apache.flink.table.functions.utils.{TableSqlFunction, UserDefinedFunctionUtils}
 import org.apache.flink.table.plan.schema.TypedFlinkTableFunction
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 
 import org.apache.calcite.plan.RelOptUtil
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
index 8657e6c..2f36241 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
@@ -20,11 +20,11 @@ package org.apache.flink.table.plan.util
 import org.apache.flink.annotation.Experimental
 import org.apache.flink.configuration.ConfigOption
 import org.apache.flink.configuration.ConfigOptions.key
-import org.apache.flink.table.api.window.TimeWindow
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.logical.{LogicalWindow, SessionGroupWindow}
 import org.apache.flink.table.plan.util.AggregateUtil.isRowtimeAttribute
-import org.apache.flink.table.runtime.window.triggers._
+import org.apache.flink.table.runtime.operators.window.TimeWindow
+import org.apache.flink.table.runtime.operators.window.triggers._
 import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 
 import java.lang.{Boolean, Long}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
index c04da62..7d68b10 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
@@ -23,7 +23,7 @@ import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.calcite.{FlinkTypeFactory, RelTimeIndicatorConverter}
 import org.apache.flink.table.codegen.{CodeGenUtils, CodeGeneratorContext, ExprCodeGenerator, ExpressionReducer, FunctionCodeGenerator}
 import org.apache.flink.table.dataformat.{BaseRow, JoinedRow}
-import org.apache.flink.table.generated.GeneratedFunction
+import org.apache.flink.table.runtime.generated.GeneratedFunction
 import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
 import org.apache.flink.table.types.logical.RowType
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/DataStreamTableSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/DataStreamTableSink.scala
index 1e76010..78411ff 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/DataStreamTableSink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/DataStreamTableSink.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.table.api.{Table, TableException}
 import org.apache.flink.table.operations.QueryOperation
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 
 /**
   * A [[DataStreamTableSink]] specifies how to emit a [[Table]] to an DataStream[T]
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala
index 019b5d0..7454692 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sinks/TableSinkUtils.scala
@@ -18,14 +18,14 @@
 
 package org.apache.flink.table.sinks
 
-import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.api.ValidationException
 import org.apache.flink.table.operations.CatalogSinkModifyOperation
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.PlannerTypeUtils
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils
 
 import java.util.{List => JList}
 
-import collection.JavaConversions._
+import scala.collection.JavaConversions._
 
 object TableSinkUtils {
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala
index 7a750ee..d4f3766 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/sources/TableSourceUtil.scala
@@ -25,8 +25,8 @@ import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.expressions.utils.ApiExpressionUtils.{typeLiteral, unresolvedCall, valueLiteral}
 import org.apache.flink.table.expressions.{ResolvedFieldReference, RexNodeConverter}
 import org.apache.flink.table.functions.BuiltInFunctionDefinitions
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.{LogicalType, TimestampKind, TimestampType, TinyIntType}
 import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
 import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
index 10155d0..96a01c6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.typeutils
 
 import org.apache.flink.table.types.logical.LogicalTypeRoot._
 import org.apache.flink.table.types.logical.{BigIntType, DoubleType, FloatType, IntType, LogicalType, SmallIntType, TinyIntType}
-import org.apache.flink.table.typeutils.TypeCheckUtils._
+import org.apache.flink.table.runtime.typeutils.TypeCheckUtils._
 
 /**
   * Utilities for type conversions.
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/LongHashJoinGeneratorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/LongHashJoinGeneratorTest.java
index 550e41d..6a94c87 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/LongHashJoinGeneratorTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/LongHashJoinGeneratorTest.java
@@ -21,10 +21,10 @@ package org.apache.flink.table.codegen;
 import org.apache.flink.api.common.functions.AbstractRichFunction;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.runtime.join.HashJoinType;
-import org.apache.flink.table.runtime.join.Int2HashJoinOperatorTest;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.join.HashJoinType;
+import org.apache.flink.table.runtime.operators.join.Int2HashJoinOperatorTest;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/SortCodeGeneratorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/SortCodeGeneratorTest.java
index 890ef80..dc05c86 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/SortCodeGeneratorTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/codegen/SortCodeGeneratorTest.java
@@ -40,14 +40,16 @@ import org.apache.flink.table.dataformat.DataFormatConverters;
 import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.TypeGetterSetters;
-import org.apache.flink.table.generated.GeneratedNormalizedKeyComputer;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.plan.util.SortUtil;
-import org.apache.flink.table.runtime.sort.BinaryInMemorySortBuffer;
+import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.InternalSerializers;
 import org.apache.flink.table.types.logical.ArrayType;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.BooleanType;
@@ -63,8 +65,6 @@ import org.apache.flink.table.types.logical.TinyIntType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
 import org.apache.flink.table.types.logical.VarBinaryType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.MutableObjectIterator;
 
@@ -508,7 +508,7 @@ public class SortCodeGeneratorTest {
 							return order ? 1 : -1;
 						}
 					} else if (t.getTypeRoot() == LogicalTypeRoot.VARBINARY) {
-						int comp = org.apache.flink.table.runtime.sort.SortUtil.compareBinary(
+						int comp = org.apache.flink.table.runtime.operators.sort.SortUtil.compareBinary(
 								(byte[]) first, (byte[]) second);
 						if (comp != 0) {
 							return order ? comp : -comp;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java
index a348e2b..8ab23b6 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.IntFi
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.LongFirstValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.ShortFirstValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.StringFirstValueAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.util.Arrays;
 import java.util.List;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java
index fc3f1f4..492b1d1 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunctionWithoutOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.IntFi
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.LongFirstValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.ShortFirstValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.StringFirstValueAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.util.Arrays;
 import java.util.List;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java
index 56d279c..ccc0581 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFun
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.LongFirstValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.ShortFirstValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.StringFirstValueWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java
index 6509286..8efe438 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunctionWithoutOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFun
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.LongFirstValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.ShortFirstValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.FirstValueWithRetractAggFunction.StringFirstValueWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java
index 4d7cb21..9c5044d 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.IntLas
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.ShortLastValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.StringLastValueAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.util.Arrays;
 import java.util.List;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java
index 58a8f3b..42360e5 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunctionWithoutOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.IntLas
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.ShortLastValueAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.StringLastValueAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.util.Arrays;
 import java.util.List;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java
index daed0f5..3d1ff60 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunc
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.LongLastValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.ShortLastValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.StringLastValueWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java
index 9e40c5b..890be99 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunctionWithoutOrderTest.java
@@ -31,7 +31,7 @@ import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunc
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.LongLastValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.ShortLastValueWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.LastValueWithRetractAggFunction.StringLastValueWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunctionTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunctionTest.java
index 50abe18..a8b84d5 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunctionTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunctionTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.table.functions.aggfunctions.MaxWithRetractAggFunction.S
 import org.apache.flink.table.functions.aggfunctions.MaxWithRetractAggFunction.StringMaxWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.MaxWithRetractAggFunction.TimeMaxWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.MaxWithRetractAggFunction.TimestampMaxWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.sql.Date;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunctionTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunctionTest.java
index 1aab445..e8511dc 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunctionTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunctionTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.table.functions.aggfunctions.MinWithRetractAggFunction.S
 import org.apache.flink.table.functions.aggfunctions.MinWithRetractAggFunction.StringMinWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.MinWithRetractAggFunction.TimeMinWithRetractAggFunction;
 import org.apache.flink.table.functions.aggfunctions.MinWithRetractAggFunction.TimestampMinWithRetractAggFunction;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 
 import java.lang.reflect.Method;
 import java.sql.Date;
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/BaseRowTestUtil.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/BaseRowTestUtil.java
index 3225243..4cf0936 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/BaseRowTestUtil.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/util/BaseRowTestUtil.java
@@ -23,10 +23,10 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryWriter;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.TypeGetterSetters;
-import org.apache.flink.table.types.InternalSerializers;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.StringUtils;
 
 import java.sql.Date;
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/ProjectionCodeGeneratorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/ProjectionCodeGeneratorTest.scala
index f6570a7..f2609e2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/ProjectionCodeGeneratorTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/ProjectionCodeGeneratorTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.codegen
 
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, GenericRow}
-import org.apache.flink.table.generated.Projection
+import org.apache.flink.table.runtime.generated.Projection
 import org.apache.flink.table.types.logical.{BigIntType, IntType, RowType}
 
 import org.junit.{Assert, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGeneratorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGeneratorTest.scala
index 4561147..47dd2b8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGeneratorTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/AggsHandlerCodeGeneratorTest.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.codegen.agg
 
 import org.apache.flink.table.api.DataTypes
 import org.apache.flink.table.dataformat.GenericRow
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore
-import org.apache.flink.table.generated.AggsHandleFunction
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore
+import org.apache.flink.table.runtime.generated.AggsHandleFunction
 import org.apache.flink.table.types.utils.TypeConversions
 
 import org.junit.{Assert, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysTest.scala
index cbead0d..67214b6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/AggWithoutKeysTest.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.table.codegen.agg.batch
 
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.{BigIntType, DoubleType, LogicalType, RowType}
 
 import org.junit.Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/BatchAggTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/BatchAggTestBase.scala
index e351d95..b214c4d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/BatchAggTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/BatchAggTestBase.scala
@@ -24,9 +24,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
 import org.apache.flink.streaming.runtime.tasks.{OneInputStreamTask, OneInputStreamTaskTestHarness}
 import org.apache.flink.table.codegen.agg.AggTestBase
 import org.apache.flink.table.dataformat.{BaseRow, BinaryString, GenericRow}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.{DoubleType, LogicalType, RowType, VarCharType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.BaseRowTestUtil
 
 import org.junit.Assert
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGeneratorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGeneratorTest.scala
index 72c3238..82ddb94 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGeneratorTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/HashAggCodeGeneratorTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.DataTypes
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.functions.aggfunctions.AvgAggFunction.IntegralAvgAggFunction
 import org.apache.flink.table.plan.util.{AggregateInfo, AggregateInfoList}
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
 import org.apache.flink.table.types.logical.{BigIntType, DoubleType, LogicalType, RowType, VarCharType}
 
 import org.apache.calcite.rel.core.AggregateCall
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGeneratorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGeneratorTest.scala
index b4a0852..d3570ab 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGeneratorTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/codegen/agg/batch/SortAggCodeGeneratorTest.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.table.codegen.agg.batch
 
 import org.apache.flink.table.dataformat.BaseRow
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.{BigIntType, DoubleType, LogicalType, RowType, VarCharType}
 
 import org.junit.Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
index d989976..19103f1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
@@ -22,8 +22,9 @@ import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.dataformat.Decimal
 import org.apache.flink.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.table.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.types.Row
+
 import org.junit.Test
 
 class DecimalTypeTest extends ExpressionTestBase {
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
index 2517d1a..30e5619 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
@@ -31,7 +31,7 @@ import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator,
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, DataFormatConverters}
 import org.apache.flink.table.planner.PlannerBase
 import org.apache.flink.table.types.DataType
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.logical.{RowType, VarCharType}
 import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.types.Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala
index f358499..eaf52c7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/RowTypeTestBase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.expressions.utils
 import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.{ObjectArrayTypeInfo, RowTypeInfo}
 import org.apache.flink.table.dataformat.Decimal
-import org.apache.flink.table.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.table.util.DateTimeTestUtil.localDate
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala
index 78b067b..997f334 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarOperatorsTestBase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.expressions.utils
 import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.dataformat.Decimal
-import org.apache.flink.table.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.table.util.DateTimeTestUtil._
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarTypesTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarTypesTestBase.scala
index 0b80ee6..1b010f5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarTypesTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/expressions/utils/ScalarTypesTestBase.scala
@@ -21,7 +21,8 @@ package org.apache.flink.table.expressions.utils
 import org.apache.flink.api.common.typeinfo.{PrimitiveArrayTypeInfo, Types}
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.dataformat.Decimal
-import org.apache.flink.table.typeutils.{DecimalTypeInfo, TimeIntervalTypeInfo}
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
 import org.apache.flink.table.util.DateTimeTestUtil._
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableSourceTest.scala
index 41b2f68..454bc75 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableSourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/TableSourceTest.scala
@@ -22,10 +22,9 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, LocalTimeTypeInfo, T
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.{DataTypes, TableSchema, Types, ValidationException}
 import org.apache.flink.table.expressions.utils.Func1
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter
 import org.apache.flink.table.sources.TableSource
-import org.apache.flink.table.types.{DataType, TypeInfoDataTypeConverter}
-import org.apache.flink.table.util._
-import org.apache.flink.table.types.TypeInfoDataTypeConverter
+import org.apache.flink.table.types.DataType
 import org.apache.flink.table.util.{TestPartitionableTableSource, _}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateTestBase.scala
index 0b8c8ed..5bef16e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/AggregateTestBase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{TableException, Types}
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.{VarSum1AggFunction, VarSum2AggFunction}
-import org.apache.flink.table.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.table.util.{BatchTableTestUtil, TableTestBase}
 
 import org.junit.Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniquenessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniquenessTest.scala
index 6929247..04e7ecb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniquenessTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnUniquenessTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.metadata
 
 import org.apache.flink.table.plan.nodes.calcite.{LogicalExpand, LogicalRank}
 import org.apache.flink.table.plan.util.ExpandUtil
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
 
 import com.google.common.collect.ImmutableList
 import org.apache.calcite.rel.`type`.RelDataTypeFieldImpl
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdHandlerTestBase.scala
index c1627a6..6d7587b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdHandlerTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdHandlerTestBase.scala
@@ -40,7 +40,7 @@ import org.apache.flink.table.plan.stream.sql.join.TestTemporalTable
 import org.apache.flink.table.plan.util.AggregateUtil.transformToStreamAggregateInfoList
 import org.apache.flink.table.plan.util._
 import org.apache.flink.table.planner.PlannerContext
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType, VariableRankRange}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType, VariableRankRange}
 import org.apache.flink.table.types.AtomicDataType
 import org.apache.flink.table.types.logical.{BigIntType, DoubleType, IntType, LogicalType, TimestampKind, TimestampType, VarCharType}
 import org.apache.flink.table.util.CountAggFunction
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala
index c62a5c0..245f8b0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/FlinkRelMdModifiedMonotonicityTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.metadata
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable
 import org.apache.flink.table.plan.`trait`.RelModifiedMonotonicity
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalRank
-import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankType}
+import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
 
 import org.apache.calcite.rel.RelCollations
 import org.apache.calcite.rel.`type`.RelDataTypeFieldImpl
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataTestUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataTestUtil.scala
index bf7b3a9..759fd32 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataTestUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/MetadataTestUtil.scala
@@ -24,9 +24,9 @@ import org.apache.flink.table.api.TableSchema
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.schema.DataStreamTable
 import org.apache.flink.table.plan.stats.{ColumnStats, FlinkStatistic, TableStats}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.types.logical.{BigIntType, IntType, LogicalType, TimestampKind, TimestampType, VarCharType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.apache.calcite.jdbc.CalciteSchema
 import org.apache.calcite.schema.SchemaPlus
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
index 5b13415..d53c6335 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
@@ -28,10 +28,10 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.{TableConfig, TableSchema, Types}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.sinks.{AppendStreamTableSink, StreamTableSink, TableSink}
 import org.apache.flink.table.sources.StreamTableSource
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.{TableTestBase, TableTestUtil, TestingTableEnvironment}
 
 import org.junit.Assert.assertEquals
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
index 4b207f4..166ddcc 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.{DataTypes, TableSchema, Types}
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.types.TypeInfoDataTypeConverter
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter
 import org.apache.flink.table.util.{TableConfigUtils, TableTestBase, TestNestedProjectableTableSource, TestProjectableTableSource}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
index 527e0e7..885ae81 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{TableException, Types, ValidationException}
-import org.apache.flink.table.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.table.util.{StreamTableTestUtil, TableTestBase}
 
 import org.junit.Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/LookupJoinTest.scala
index 1666fa0..908cd95 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/join/LookupJoinTest.scala
@@ -25,9 +25,9 @@ import org.apache.flink.table.api._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.dataformat.{BaseRow, BinaryString}
 import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.sources._
 import org.apache.flink.table.types.logical.{IntType, TimestampType, VarCharType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.TableTestBase
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeTestBase.scala
index d3fed41..79a3fbb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/util/RexNodeTestBase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.util
 
 import org.apache.flink.table.api.DataTypes
 import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkTypeSystem}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex.{RexBuilder, RexNode, RexProgram, RexProgramBuilder}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase.scala
index 77c82a8..dd8840c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CorrelateITCase.scala
@@ -27,12 +27,12 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.table.dataformat.BinaryString
 import org.apache.flink.table.expressions.utils.{Func1, Func18, RichFunc2}
 import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.JavaTableFunc0
 import org.apache.flink.table.runtime.utils.TestData._
 import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils.{MyPojo, MyPojoFunc}
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo
 import org.apache.flink.table.util.DateTimeTestUtil._
 import org.apache.flink.table.util._
 import org.apache.flink.types.Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
index 82b5ecb..c3c1447 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
@@ -23,9 +23,9 @@ import org.apache.flink.table.api.DataTypes
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.logical.{DecimalType, LogicalType}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableSourceITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableSourceITCase.scala
index fcda2f9..b0276ae 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableSourceITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableSourceITCase.scala
@@ -21,9 +21,9 @@ package org.apache.flink.table.runtime.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.{DataTypes, TableSchema, Types}
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.{BatchTestBase, TestData}
-import org.apache.flink.table.types.TypeInfoDataTypeConverter
 import org.apache.flink.table.util.{TestFilterableTableSource, TestNestedProjectableTableSource, TestPartitionableTableSource, TestProjectableTableSource, TestTableSources}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
index 43b2108..489733b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
@@ -20,11 +20,11 @@ package org.apache.flink.table.runtime.batch.sql
 
 import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.dataformat.BinaryString.fromString
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.{binaryRow, row}
 import org.apache.flink.table.runtime.utils.TestData._
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.junit._
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
index b544835..5d238a0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
@@ -23,10 +23,10 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.runtime.batch.sql.join.JoinITCaseHelper.disableOtherJoinOpForJoin
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.TestData._
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo
 
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
index 216dca8..57cf376 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
@@ -25,8 +25,8 @@ import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo}
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.expressions.utils.FuncWithOpen
-import org.apache.flink.table.runtime.CodeGenOperatorFactory
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.TestData._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
index aa345d1..27721a4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
@@ -24,9 +24,9 @@ import org.apache.flink.table.api.{DataTypes, Table}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.{BatchTableEnvUtil, BatchTestBase}
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
-import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.logical.{DecimalType, LogicalType}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
index cfa327f..01401a0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/AbstractTwoInputStreamOperatorWithTTLTest.scala
@@ -26,14 +26,12 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
 import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.runtime.harness.HarnessTestBase.TestingBaseRowKeySelector
-import org.apache.flink.table.runtime.join.temporal.BaseTwoInputStreamOperatorWithStateRetention
+import org.apache.flink.table.runtime.operators.join.temporal.BaseTwoInputStreamOperatorWithStateRetention
 import org.apache.flink.table.runtime.util.StreamRecordUtils.record
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.HEAP_BACKEND
-import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+
 import org.hamcrest.MatcherAssert.assertThat
 import org.hamcrest.{Description, TypeSafeMatcher}
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
 import org.junit.{After, Before, Test}
 
 import java.lang.{Long => JLong}
@@ -42,8 +40,7 @@ import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
 
 /**
-  * Tests for the
-  * [[org.apache.flink.table.runtime.join.temporal.BaseTwoInputStreamOperatorWithStateRetention]].
+  * Tests for the [[BaseTwoInputStreamOperatorWithStateRetention]].
   */
 class AbstractTwoInputStreamOperatorWithTTLTest
   extends HarnessTestBase(HEAP_BACKEND) {
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
index 15811e7..702d225 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/AggregateITCase.scala
@@ -27,13 +27,13 @@ import org.apache.flink.table.api.scala._
 import org.apache.flink.table.functions.aggfunctions.{ConcatWithRetractAggFunction, ConcatWsWithRetractAggFunction}
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.VarSumAggFunction
 import org.apache.flink.table.runtime.batch.sql.agg.{MyPojoAggFunction, VarArgsAggFunction}
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo
 import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase.AggMode
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.MiniBatchMode
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.table.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset
 import org.apache.flink.table.runtime.utils.UserDefinedFunctionTestUtils._
-import org.apache.flink.table.runtime.utils.{BatchTestBase, StreamingWithAggTestBase, TestData, TestingRetractSink}
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo
+import org.apache.flink.table.runtime.utils.{StreamingWithAggTestBase, TestData, TestingRetractSink}
 import org.apache.flink.table.util.DateTimeTestUtil.{localDate, localDateTime, localTime => mLocalTime}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
index 7127ac9..b2159ff 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
@@ -24,9 +24,9 @@ import org.apache.flink.api.scala._
 import org.apache.flink.api.scala.typeutils.Types
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.dataformat.{BaseRow, GenericRow}
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.runtime.utils.{StreamingTestBase, TestData, TestSinkUtil, TestingAppendBaseRowSink, TestingAppendSink, TestingAppendTableSink}
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.types.Row
 
 import org.junit.Assert._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
index e0b9a31..d6a8ccf 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
@@ -24,7 +24,7 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.table.runtime.utils.{TestingRetractTableSink, TestingUpsertTableSink, _}
-import org.apache.flink.table.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 import org.apache.flink.types.Row
 
 import org.junit.Assert._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/ValuesITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/ValuesITCase.scala
index 20be2f4..1465970 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/ValuesITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/ValuesITCase.scala
@@ -21,9 +21,9 @@ package org.apache.flink.table.runtime.stream.sql
 import org.apache.flink.streaming.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.dataformat.BaseRow
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.runtime.utils.{StreamingTestBase, TestingAppendBaseRowSink}
 import org.apache.flink.table.types.logical.IntType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.junit.Assert._
 import org.junit.Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
index e31971a..3edbc8f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
@@ -31,9 +31,9 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
 import org.apache.flink.table.planner.PlannerBase
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.runtime.utils.BatchAbstractTestBase.DEFAULT_PARALLELISM
 import org.apache.flink.table.types.logical.{BigIntType, LogicalType}
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.{BaseRowTestUtil, TableTestUtil, TestingTableEnvironment}
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestSink.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestSink.scala
index 8067817..423f50e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestSink.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestSink.scala
@@ -31,10 +31,10 @@ import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink}
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.dataformat.{BaseRow, DataFormatConverters, GenericRow}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.sinks._
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.types.utils.TypeConversions
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.util.BaseRowTestUtil
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
index 0cf88a0..8502fd2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala
@@ -29,10 +29,10 @@ import org.apache.flink.streaming.api.functions.source.FromElementsFunction
 import org.apache.flink.streaming.api.scala.DataStream
 import org.apache.flink.table.api.scala.StreamTableEnvironment
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, BinaryRowWriter, BinaryString}
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.types.logical.RowType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 
 import org.junit.runners.Parameterized
 import org.junit.{After, Assert, Before}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TableUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TableUtil.scala
index ac54215..3fa4daf 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TableUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TableUtil.scala
@@ -22,8 +22,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.internal.TableImpl
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.sinks.{CollectRowTableSink, CollectTableSink}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.types.logical.TimestampType
 import org.apache.flink.table.util.TableTestUtil
 import org.apache.flink.types.Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
index ed73b52..f10bdbc 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/TestSinkUtil.scala
@@ -21,13 +21,15 @@ package org.apache.flink.table.runtime.utils
 import org.apache.flink.table.api.{Table, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.GenericRow
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.runtime.utils.JavaPojos.Pojo1
 import org.apache.flink.table.sinks.TableSink
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.util.TableTestUtil
 import org.apache.flink.types.Row
 import org.apache.flink.util.StringUtils
+
 import org.apache.calcite.avatica.util.DateTimeUtils
+
 import java.sql.{Date, Time, Timestamp}
 import java.util.{Calendar, TimeZone}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/AvgAggFunction.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/AvgAggFunction.scala
index 468ddce..5faae58 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/AvgAggFunction.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/AvgAggFunction.scala
@@ -23,8 +23,8 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.calcite.FlinkTypeSystem
 import org.apache.flink.table.functions.AggregateFunction
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo
 import org.apache.flink.table.types.logical.DecimalType
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo
 
 import java.lang.{Iterable => JIterable}
 import java.math.{BigDecimal, BigInteger, MathContext}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
index 08b5506..654c78b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
@@ -50,7 +50,7 @@ import org.apache.flink.table.planner.PlannerBase
 import org.apache.flink.table.runtime.utils.{TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink}
 import org.apache.flink.table.sinks._
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.table.typeutils.FieldInfoUtils
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java
index d45da77..b70cc4e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/AbstractBinaryWriter.java
@@ -20,10 +20,10 @@ package org.apache.flink.table.dataformat;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 import java.io.IOException;
 import java.io.OutputStream;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java
index fbda19c..acf5adc 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArray.java
@@ -20,8 +20,8 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import java.lang.reflect.Array;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java
index 42a2fd0..ad97e35 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryArrayWriter.java
@@ -18,8 +18,8 @@
 package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.SegmentsUtil;
 
 /**
  * Writer for binary array. See {@link BinaryArray}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java
index fe374eb..2a6ccc4 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryFormat.java
@@ -18,7 +18,7 @@
 package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 /**
  * Binary format that in {@link MemorySegment}s.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java
index b815c0f..390e43b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryGeneric.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.dataformat;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.util.FlinkRuntimeException;
 import org.apache.flink.util.InstantiationUtil;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java
index 8da32db..8660fee 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryMap.java
@@ -20,13 +20,13 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.flink.table.types.ClassLogicalTypeConverter.getInternalClassForType;
+import static org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getInternalClassForType;
 import static org.apache.flink.util.Preconditions.checkArgument;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java
index 851f355..bc98c43 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRow.java
@@ -19,10 +19,10 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import java.nio.ByteOrder;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRowWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRowWriter.java
index 8c5f656..cc3a1ea 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRowWriter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryRowWriter.java
@@ -18,7 +18,7 @@
 package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 /**
  * Writer for {@link BinaryRow}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java
index 79afae6..f3db1eb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryString.java
@@ -20,9 +20,9 @@ package org.apache.flink.table.dataformat;
 import org.apache.flink.api.common.typeinfo.TypeInfo;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfoFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.runtime.util.StringUtf8Utils;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfoFactory;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import javax.annotation.Nonnull;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java
index b6525f2..ecc6152 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryStringUtil.java
@@ -18,8 +18,8 @@
 package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.runtime.util.StringUtf8Utils;
-import org.apache.flink.table.util.SegmentsUtil;
 import org.apache.flink.table.utils.EncodingUtils;
 
 import java.math.BigDecimal;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java
index 307d502..8163f57 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/BinaryWriter.java
@@ -18,11 +18,11 @@
 package org.apache.flink.table.dataformat;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 /**
  * Writer to write a composite data format, like row, array.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java
index cf7d882..de29de7 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/DataFormatConverters.java
@@ -30,19 +30,19 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.CollectionDataType;
 import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.InternalSerializers;
 import org.apache.flink.table.types.KeyValueDataType;
-import org.apache.flink.table.types.LogicalTypeDataTypeConverter;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LegacyTypeInformationType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
 import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 import org.apache.flink.types.Row;
 
 import org.apache.commons.lang3.ArrayUtils;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/Decimal.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/Decimal.java
index 5a71eec..2fbbbc1 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/Decimal.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/Decimal.java
@@ -19,9 +19,9 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.api.common.typeinfo.TypeInfo;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfoFactory;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.typeutils.DecimalTypeInfoFactory;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java
index 36fcf44..252eec9 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataformat/NestedRow.java
@@ -19,7 +19,7 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 import static org.apache.flink.table.dataformat.BinaryRow.calculateBitSetWidthInBytes;
 import static org.apache.flink.util.Preconditions.checkArgument;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/NullAwareMapIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/NullAwareMapIterator.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/NullAwareMapIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/NullAwareMapIterator.java
index e0335ff..e7bab92 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/NullAwareMapIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/NullAwareMapIterator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import java.util.Iterator;
 import java.util.Map;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerKeyStateDataViewStore.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerKeyStateDataViewStore.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerKeyStateDataViewStore.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerKeyStateDataViewStore.java
index d215a03..b92b075 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerKeyStateDataViewStore.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerKeyStateDataViewStore.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.ListState;
@@ -25,6 +25,8 @@ import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.table.dataview.ListViewTypeInfo;
+import org.apache.flink.table.dataview.MapViewTypeInfo;
 
 /**
  * Default implementation of StateDataViewStore that currently forwards state registration
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerWindowStateDataViewStore.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerWindowStateDataViewStore.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerWindowStateDataViewStore.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerWindowStateDataViewStore.java
index fcfeb68..3bdc448 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/PerWindowStateDataViewStore.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/PerWindowStateDataViewStore.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.ListState;
@@ -30,6 +30,8 @@ import org.apache.flink.runtime.state.KeyedStateBackend;
 import org.apache.flink.runtime.state.internal.InternalListState;
 import org.apache.flink.runtime.state.internal.InternalMapState;
 import org.apache.flink.runtime.state.internal.InternalValueState;
+import org.apache.flink.table.dataview.ListViewTypeInfo;
+import org.apache.flink.table.dataview.MapViewTypeInfo;
 
 /**
  * An implementation of StateDataViewStore for window aggregates which forward the state
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataView.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataView.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataView.java
index 286a47e..a54dd3d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataView.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataView.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.table.api.dataview.DataView;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataViewStore.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataViewStore.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataViewStore.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataViewStore.java
index aa1fb2f..7a93271 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateDataViewStore.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateDataViewStore.java
@@ -16,9 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.table.dataview.ListViewTypeInfo;
+import org.apache.flink.table.dataview.MapViewTypeInfo;
 
 /**
  * This interface contains methods for registering {@link StateDataView} with a managed store.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateListView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateListView.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateListView.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateListView.java
index 1908a19..34faf42 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateListView.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateListView.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.runtime.state.internal.InternalListState;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateMapView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateMapView.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java
index c7f57c5..c7f2686 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/dataview/StateMapView.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/dataview/StateMapView.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.dataview;
+package org.apache.flink.table.runtime.dataview;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.ValueState;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlLikeChainChecker.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlLikeChainChecker.java
index 361e0ba..8c7bb0c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlLikeChainChecker.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlLikeChainChecker.java
@@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.functions;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 import org.apache.commons.lang3.ArrayUtils;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/AggsHandleFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/AggsHandleFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/AggsHandleFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/AggsHandleFunction.java
index 62fb99e..953667e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/AggsHandleFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/AggsHandleFunction.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.functions.AggregateFunction;
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.dataview.StateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
 
 /**
  * The base class for handling aggregate functions.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/CompileUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/CompileUtils.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/CompileUtils.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/CompileUtils.java
index afeef4b..8de3965 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/CompileUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/CompileUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.api.java.tuple.Tuple2;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedAggsHandleFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedAggsHandleFunction.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedAggsHandleFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedAggsHandleFunction.java
index 8b9372a..2839809 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedAggsHandleFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedAggsHandleFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link AggsHandleFunction}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedClass.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedClass.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedClass.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedClass.java
index 97495e5..45c2992 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedClass.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedClass.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import java.io.Serializable;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedCollector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedCollector.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedCollector.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedCollector.java
index 3e36d12..c360e77 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedCollector.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedCollector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedFunction.java
index 69163f7..67adfe0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.functions.Function;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedHashFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedHashFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedHashFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedHashFunction.java
index 914d0bc..ac4c0b6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedHashFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedHashFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link HashFunction}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedInput.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedInput.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedInput.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedInput.java
index 96ddc4a..0cce73b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedInput.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedInput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.io.InputFormat;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedJoinCondition.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedJoinCondition.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedJoinCondition.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedJoinCondition.java
index 60ff809..bf53de8 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedJoinCondition.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedJoinCondition.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link JoinCondition}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNamespaceAggsHandleFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNamespaceAggsHandleFunction.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNamespaceAggsHandleFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNamespaceAggsHandleFunction.java
index 5a9ef8b..f33b044 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNamespaceAggsHandleFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNamespaceAggsHandleFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link NamespaceAggsHandleFunction}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNormalizedKeyComputer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNormalizedKeyComputer.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNormalizedKeyComputer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNormalizedKeyComputer.java
index e524d40..0ef2750 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedNormalizedKeyComputer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedNormalizedKeyComputer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link NormalizedKeyComputer}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedOperator.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedOperator.java
index e0316ed..e24aeca 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.streaming.api.operators.StreamOperator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedProjection.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedProjection.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedProjection.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedProjection.java
index d43b5f3..306a783 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedProjection.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedProjection.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link Projection}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordComparator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordComparator.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordComparator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordComparator.java
index 2db63cc..d266e8b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordComparator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordComparator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link RecordComparator}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordEqualiser.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordEqualiser.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordEqualiser.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordEqualiser.java
index ecab896..d8fc2bb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedRecordEqualiser.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedRecordEqualiser.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 /**
  * Describes a generated {@link RecordEqualiser}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedResultFuture.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedResultFuture.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedResultFuture.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedResultFuture.java
index e791426..c8ba5f98 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/GeneratedResultFuture.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/GeneratedResultFuture.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.streaming.api.functions.async.ResultFuture;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/HashFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/HashFunction.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/HashFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/HashFunction.java
index 19d1e2f..c5d8441 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/HashFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/HashFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/JoinCondition.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/JoinCondition.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/JoinCondition.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/JoinCondition.java
index 212ed5d..4f12d1f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/JoinCondition.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/JoinCondition.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.functions.RichFunction;
 import org.apache.flink.table.dataformat.BaseRow;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NamespaceAggsHandleFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NamespaceAggsHandleFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NamespaceAggsHandleFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NamespaceAggsHandleFunction.java
index 4e43101..12c14de 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NamespaceAggsHandleFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NamespaceAggsHandleFunction.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.dataview.StateDataViewStore;
+import org.apache.flink.table.runtime.dataview.StateDataViewStore;
 
 /**
  * The base class for handling aggregate functions.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NormalizedKeyComputer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NormalizedKeyComputer.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NormalizedKeyComputer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NormalizedKeyComputer.java
index d960a74..fae44a5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/NormalizedKeyComputer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/NormalizedKeyComputer.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.sort.BinaryInMemorySortBuffer;
+import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer;
 
 /**
  * Normalized key computer for {@link BinaryInMemorySortBuffer}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/Projection.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/Projection.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/Projection.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/Projection.java
index 3d7bdf1..e37c23e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/Projection.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/Projection.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordComparator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordComparator.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordComparator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordComparator.java
index 5331bcf..3d6ee36 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordComparator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordComparator.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.sort.BinaryInMemorySortBuffer;
+import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer;
 
 import java.io.Serializable;
 import java.util.Comparator;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordEqualiser.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordEqualiser.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordEqualiser.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordEqualiser.java
index 9db503e..1d045ae 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/generated/RecordEqualiser.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/generated/RecordEqualiser.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashPartition.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashPartition.java
index 9ba4cc8..22bc501 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashPartition.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashPartition.java
@@ -31,10 +31,10 @@ import org.apache.flink.runtime.memory.AbstractPagedInputView;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
 import org.apache.flink.table.runtime.util.RowIterator;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MathUtils;
 
 import java.io.EOFException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashTable.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashTable.java
index eb96bdf..8297418 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashTable.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BinaryHashTable.java
@@ -29,16 +29,16 @@ import org.apache.flink.runtime.operators.util.BitSet;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.util.BinaryRowUtil;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.generated.Projection;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.generated.Projection;
 import org.apache.flink.table.runtime.io.BinaryRowChannelInputViewIterator;
 import org.apache.flink.table.runtime.io.ChannelWithMeta;
-import org.apache.flink.table.runtime.join.HashJoinType;
-import org.apache.flink.table.runtime.join.NullAwareJoinHelper;
+import org.apache.flink.table.runtime.operators.join.HashJoinType;
+import org.apache.flink.table.runtime.operators.join.NullAwareJoinHelper;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.runtime.util.RowIterator;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MathUtils;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BuildSideIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BuildSideIterator.java
index f43803b..d8d8302 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BuildSideIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BuildSideIterator.java
@@ -21,8 +21,8 @@ package org.apache.flink.table.runtime.hashtable;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.operators.util.BitSet;
 import org.apache.flink.table.dataformat.BinaryRow;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.RowIterator;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHashPartition.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHashPartition.java
index 7d54945..17e8902 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHashPartition.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHashPartition.java
@@ -30,9 +30,9 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.AbstractPagedInputView;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.runtime.util.RowIterator;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MathUtils;
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHybridHashTable.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHybridHashTable.java
index c65d67d..117d5a6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHybridHashTable.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/LongHybridHashTable.java
@@ -28,8 +28,8 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
 import org.apache.flink.table.runtime.io.ChannelWithMeta;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MathUtils;
 
 import java.io.EOFException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/io/BinaryRowChannelInputViewIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/io/BinaryRowChannelInputViewIterator.java
index 4920334..2648eba 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/io/BinaryRowChannelInputViewIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/io/BinaryRowChannelInputViewIterator.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.io;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.ChannelReaderInputView;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.EOFException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BaseRowKeySelector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BaseRowKeySelector.java
index 2ad9e5d..3e49453 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BaseRowKeySelector.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BaseRowKeySelector.java
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.keyselector;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * BaseRowKeySelector takes an BaseRow and extracts the deterministic key for the BaseRow.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BinaryRowKeySelector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BinaryRowKeySelector.java
index 9d0d9ee..33bf9af 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BinaryRowKeySelector.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/BinaryRowKeySelector.java
@@ -20,9 +20,9 @@ package org.apache.flink.table.runtime.keyselector;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedProjection;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * A KeySelector which will extract key from BaseRow. The key type is BinaryRow.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/NullBinaryRowKeySelector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/NullBinaryRowKeySelector.java
index 7694a84..98791c5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/NullBinaryRowKeySelector.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/keyselector/NullBinaryRowKeySelector.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.runtime.keyselector;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.util.BinaryRowUtil;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * A utility class which key is always empty no matter what the input row is.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/AbstractProcessStreamOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/AbstractProcessStreamOperator.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/AbstractProcessStreamOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/AbstractProcessStreamOperator.java
index ef1aef3..80a715e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/AbstractProcessStreamOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/AbstractProcessStreamOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime;
+package org.apache.flink.table.runtime.operators;
 
 import org.apache.flink.streaming.api.TimerService;
 import org.apache.flink.streaming.api.operators.InternalTimerService;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/CodeGenOperatorFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java
similarity index 78%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/CodeGenOperatorFactory.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java
index 61d6fb8..d6e6b62 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/CodeGenOperatorFactory.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/CodeGenOperatorFactory.java
@@ -1,12 +1,13 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *    http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime;
+package org.apache.flink.table.runtime.operators;
 
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
@@ -25,7 +26,7 @@ import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.table.generated.GeneratedClass;
+import org.apache.flink.table.runtime.generated.GeneratedClass;
 
 /**
  * Stream operator factory for code gen operator.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/TableStreamOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/TableStreamOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java
index ddac52d..744c278 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/TableStreamOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime;
+package org.apache.flink.table.runtime.operators;
 
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMap.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMap.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java
index da4b0b7..f10a896 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMap.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMap.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
@@ -27,8 +27,8 @@ import org.apache.flink.runtime.memory.AbstractPagedInputView;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.table.dataformat.BinaryRow;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MathUtils;
 import org.apache.flink.util.MutableObjectIterator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMapSpillMemorySegmentPool.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapSpillMemorySegmentPool.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMapSpillMemorySegmentPool.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapSpillMemorySegmentPool.java
index d3088e6..72a4289 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/BytesHashMapSpillMemorySegmentPool.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapSpillMemorySegmentPool.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/GroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/GroupAggFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
index f40ea63..d7a400b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/GroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
@@ -16,21 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import static org.apache.flink.table.dataformat.util.BaseRowUtil.ACCUMULATE_MSG;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGlobalGroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGlobalGroupAggFunction.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGlobalGroupAggFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGlobalGroupAggFunction.java
index 12fc464..c6d89f6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGlobalGroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGlobalGroupAggFunction.java
@@ -16,21 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.RecordEqualiser;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGroupAggFunction.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGroupAggFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGroupAggFunction.java
index ff2d780..acd068d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchGroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchGroupAggFunction.java
@@ -16,24 +16,24 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.RecordEqualiser;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
-import org.apache.flink.table.types.InternalSerializers;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchIncrementalGroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchIncrementalGroupAggFunction.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchIncrementalGroupAggFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchIncrementalGroupAggFunction.java
index 55cedaf..142d7e2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchIncrementalGroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchIncrementalGroupAggFunction.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchLocalGroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchLocalGroupAggFunction.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchLocalGroupAggFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchLocalGroupAggFunction.java
index 95de323..d891b35 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/MiniBatchLocalGroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/MiniBatchLocalGroupAggFunction.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/RecordCounter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/RecordCounter.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/RecordCounter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/RecordCounter.java
index e73fb9e..2201bde 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/aggregate/RecordCounter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/RecordCounter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/AbstractMapBundleOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/AbstractMapBundleOperator.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/AbstractMapBundleOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/AbstractMapBundleOperator.java
index 8f2c763..02c9066 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/AbstractMapBundleOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/AbstractMapBundleOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle;
+package org.apache.flink.table.runtime.operators.bundle;
 
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.metrics.Gauge;
@@ -25,9 +25,9 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.table.runtime.bundle.trigger.BundleTrigger;
-import org.apache.flink.table.runtime.bundle.trigger.BundleTriggerCallback;
 import org.apache.flink.table.runtime.context.ExecutionContextImpl;
+import org.apache.flink.table.runtime.operators.bundle.trigger.BundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.trigger.BundleTriggerCallback;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/KeyedMapBundleOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/KeyedMapBundleOperator.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/KeyedMapBundleOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/KeyedMapBundleOperator.java
index ac6ab34..2c94bdf 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/KeyedMapBundleOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/KeyedMapBundleOperator.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle;
+package org.apache.flink.table.runtime.operators.bundle;
 
-import org.apache.flink.table.runtime.bundle.trigger.BundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.trigger.BundleTrigger;
 
 /**
  * The {@link KeyedMapBundleOperator} uses framework's key as bundle map key, thus can only be
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleFunction.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleFunction.java
index 793e137..51e004e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle;
+package org.apache.flink.table.runtime.operators.bundle;
 
 import org.apache.flink.api.common.functions.Function;
 import org.apache.flink.table.runtime.context.ExecutionContext;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperator.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperator.java
index 4d2144d..f0f8687 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/MapBundleOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperator.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle;
+package org.apache.flink.table.runtime.operators.bundle;
 
 import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.table.runtime.bundle.trigger.BundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.trigger.BundleTrigger;
 
 /**
  * The {@link MapBundleOperator} uses a {@link KeySelector} to extract bundle key, thus can be
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTrigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTrigger.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTrigger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTrigger.java
index 7b8ce67..27d8be5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTrigger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTrigger.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.apache.flink.annotation.Internal;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTriggerCallback.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTriggerCallback.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTriggerCallback.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTriggerCallback.java
index 425f8af..66ce574 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/BundleTriggerCallback.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/BundleTriggerCallback.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.apache.flink.annotation.Internal;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CoBundleTrigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CoBundleTrigger.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CoBundleTrigger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CoBundleTrigger.java
index d649ce1..c847790 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CoBundleTrigger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CoBundleTrigger.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.apache.flink.annotation.Internal;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTrigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTrigger.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTrigger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTrigger.java
index 8aa5614..7f7b0de 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTrigger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTrigger.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTrigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTrigger.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTrigger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTrigger.java
index ec623bf..380960c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTrigger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTrigger.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionHelper.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionHelper.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionHelper.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionHelper.java
index 1c554bd..f055944 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionHelper.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionHelper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.table.dataformat.BaseRow;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunction.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunction.java
index 14feaf9..820defa 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
@@ -26,7 +26,7 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
 import org.apache.flink.util.Collector;
 
-import static org.apache.flink.table.runtime.deduplicate.DeduplicateFunctionHelper.processFirstRow;
+import static org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.processFirstRow;
 
 /**
  * This function is used to deduplicate on keys and keeps only first row.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunction.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunction.java
index 68bce4c..93723d2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunction.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
-import static org.apache.flink.table.runtime.deduplicate.DeduplicateFunctionHelper.processLastRow;
+import static org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.processLastRow;
 
 /**
  * This function is used to deduplicate on keys and keeps only last row.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
index b97d1a6..443142c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunction.java
@@ -16,22 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
 
 import java.util.Map;
 
-import static org.apache.flink.table.runtime.deduplicate.DeduplicateFunctionHelper.processFirstRow;
+import static org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.processFirstRow;
 
 /**
  * This function is used to get the first row for every key partition in miniBatch mode.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
index c1f2ec4..392e97c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunction.java
@@ -16,22 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.bundle.MapBundleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.operators.bundle.MapBundleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import javax.annotation.Nullable;
 
 import java.util.Map;
 
-import static org.apache.flink.table.runtime.deduplicate.DeduplicateFunctionHelper.processLastRow;
+import static org.apache.flink.table.runtime.operators.deduplicate.DeduplicateFunctionHelper.processLastRow;
 
 /**
  * This function is used to get the last row for every key partition in miniBatch mode.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/EmitAwareCollector.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/EmitAwareCollector.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/EmitAwareCollector.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/EmitAwareCollector.java
index f9097bb..4447b8a 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/EmitAwareCollector.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/EmitAwareCollector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.util.Collector;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/FlinkJoinType.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/FlinkJoinType.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/FlinkJoinType.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/FlinkJoinType.java
index cbf6a35..7ae8976 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/FlinkJoinType.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/FlinkJoinType.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 /**
  * Join type for join.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java
index ae190cd..1199261 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.configuration.AlgorithmOptions;
 import org.apache.flink.configuration.Configuration;
@@ -29,15 +29,15 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.GeneratedProjection;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.JoinCondition;
 import org.apache.flink.table.runtime.hashtable.BinaryHashTable;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
 import org.apache.flink.table.runtime.util.RowIterator;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinType.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinType.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinType.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinType.java
index 3765551..d31f9af 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/HashJoinType.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinType.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 /**
  * Join type for hash table.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/KeyedCoProcessOperatorWithWatermarkDelay.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/KeyedCoProcessOperatorWithWatermarkDelay.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java
index a4f4f4f..252641d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/KeyedCoProcessOperatorWithWatermarkDelay.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/KeyedCoProcessOperatorWithWatermarkDelay.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
 import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/NullAwareJoinHelper.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/NullAwareJoinHelper.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/NullAwareJoinHelper.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/NullAwareJoinHelper.java
index 45715de..601c879 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/NullAwareJoinHelper.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/NullAwareJoinHelper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BinaryRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/OuterJoinPaddingUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/OuterJoinPaddingUtil.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/OuterJoinPaddingUtil.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/OuterJoinPaddingUtil.java
index 4242730..4baa81d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/OuterJoinPaddingUtil.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/OuterJoinPaddingUtil.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.GenericRow;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamJoin.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/ProcTimeBoundedStreamJoin.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamJoin.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/ProcTimeBoundedStreamJoin.java
index 3d56112..97347d7 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamJoin.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/ProcTimeBoundedStreamJoin.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * The function to execute processing time bounded stream inner-join.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/RowTimeBoundedStreamJoin.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/RowTimeBoundedStreamJoin.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/RowTimeBoundedStreamJoin.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/RowTimeBoundedStreamJoin.java
index 506a600..e7a2682 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/RowTimeBoundedStreamJoin.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/RowTimeBoundedStreamJoin.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * The function to execute row(event) time bounded stream inner-join.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeFullOuterJoinIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeFullOuterJoinIterator.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeFullOuterJoinIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeFullOuterJoinIterator.java
index 4e939ff..cb8242f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeFullOuterJoinIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeFullOuterJoinIterator.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.Closeable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeInnerJoinIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeInnerJoinIterator.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeInnerJoinIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeInnerJoinIterator.java
index 71a1fdb..8ebe572 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeInnerJoinIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeInnerJoinIterator.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIterator.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIterator.java
index 9667027..c2d3bf5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinIterator.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.Closeable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinOperator.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinOperator.java
index 7afbe65..df032cb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinOperator.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.configuration.Configuration;
@@ -31,19 +31,19 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.GeneratedNormalizedKeyComputer;
-import org.apache.flink.table.generated.GeneratedProjection;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
-import org.apache.flink.table.runtime.sort.BinaryExternalSorter;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.operators.sort.BinaryExternalSorter;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeOneSideOuterJoinIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeOneSideOuterJoinIterator.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeOneSideOuterJoinIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeOneSideOuterJoinIterator.java
index 7846321..a045c76 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/SortMergeOneSideOuterJoinIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeOneSideOuterJoinIterator.java
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/TimeBoundedStreamJoin.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/TimeBoundedStreamJoin.java
index 1cef678..38e39bf 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/TimeBoundedStreamJoin.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
 import org.apache.flink.api.common.state.MapState;
@@ -30,8 +30,8 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java
index d6e9468..00116a5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinRunner.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.lookup;
+package org.apache.flink.table.runtime.operators.join.lookup;
 
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -30,10 +30,10 @@ import org.apache.flink.table.dataformat.DataFormatConverters;
 import org.apache.flink.table.dataformat.DataFormatConverters.RowConverter;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedFunction;
-import org.apache.flink.table.generated.GeneratedResultFuture;
 import org.apache.flink.table.runtime.collector.TableFunctionResultFuture;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.generated.GeneratedResultFuture;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.types.Row;
 
 import javax.annotation.Nullable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinWithCalcRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinWithCalcRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java
index 925d52e..75b7b25 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/AsyncLookupJoinWithCalcRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.lookup;
+package org.apache.flink.table.runtime.operators.join.lookup;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
@@ -26,10 +26,10 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.async.AsyncFunction;
 import org.apache.flink.streaming.api.functions.async.ResultFuture;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
-import org.apache.flink.table.generated.GeneratedResultFuture;
 import org.apache.flink.table.runtime.collector.TableFunctionResultFuture;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.generated.GeneratedResultFuture;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/DelegatingResultFuture.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/DelegatingResultFuture.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java
index 9f24aeb..cf87463 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/DelegatingResultFuture.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/DelegatingResultFuture.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.lookup;
+package org.apache.flink.table.runtime.operators.join.lookup;
 
 import org.apache.flink.streaming.api.functions.async.ResultFuture;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java
index 58bcbe7..465c958 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.lookup;
+package org.apache.flink.table.runtime.operators.join.lookup;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
@@ -25,9 +25,9 @@ import org.apache.flink.streaming.api.functions.ProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedCollector;
-import org.apache.flink.table.generated.GeneratedFunction;
 import org.apache.flink.table.runtime.collector.TableFunctionCollector;
+import org.apache.flink.table.runtime.generated.GeneratedCollector;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
 import org.apache.flink.util.Collector;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinWithCalcRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinWithCalcRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java
index 1670258..1329970 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/lookup/LookupJoinWithCalcRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.lookup;
+package org.apache.flink.table.runtime.operators.join.lookup;
 
 import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedCollector;
-import org.apache.flink.table.generated.GeneratedFunction;
 import org.apache.flink.table.runtime.collector.TableFunctionCollector;
+import org.apache.flink.table.runtime.generated.GeneratedCollector;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
 import org.apache.flink.util.Collector;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/AbstractStreamingJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/AbstractStreamingJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java
index 588c1d7..ead4b5b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/AbstractStreamingJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream;
+package org.apache.flink.table.runtime.operators.join.stream;
 
 import org.apache.flink.api.common.functions.AbstractRichFunction;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -26,13 +26,13 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.runtime.join.NullAwareJoinHelper;
-import org.apache.flink.table.runtime.join.stream.state.JoinInputSideSpec;
-import org.apache.flink.table.runtime.join.stream.state.JoinRecordStateView;
-import org.apache.flink.table.runtime.join.stream.state.OuterJoinRecordStateView;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.join.NullAwareJoinHelper;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateView;
+import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateView;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.IterableIterator;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java
index 2cc9896..9463029 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingJoinOperator.java
@@ -16,20 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream;
+package org.apache.flink.table.runtime.operators.join.stream;
 
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.runtime.join.stream.state.JoinInputSideSpec;
-import org.apache.flink.table.runtime.join.stream.state.JoinRecordStateView;
-import org.apache.flink.table.runtime.join.stream.state.JoinRecordStateViews;
-import org.apache.flink.table.runtime.join.stream.state.OuterJoinRecordStateView;
-import org.apache.flink.table.runtime.join.stream.state.OuterJoinRecordStateViews;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateView;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateViews;
+import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateView;
+import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateViews;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * Streaming unbounded Join operator which supports INNER/LEFT/RIGHT/FULL JOIN.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingSemiAntiJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingSemiAntiJoinOperator.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingSemiAntiJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingSemiAntiJoinOperator.java
index 6a8a278..fc45876 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/StreamingSemiAntiJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/StreamingSemiAntiJoinOperator.java
@@ -16,18 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream;
+package org.apache.flink.table.runtime.operators.join.stream;
 
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.runtime.join.stream.state.JoinInputSideSpec;
-import org.apache.flink.table.runtime.join.stream.state.JoinRecordStateView;
-import org.apache.flink.table.runtime.join.stream.state.JoinRecordStateViews;
-import org.apache.flink.table.runtime.join.stream.state.OuterJoinRecordStateView;
-import org.apache.flink.table.runtime.join.stream.state.OuterJoinRecordStateViews;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateView;
+import org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateViews;
+import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateView;
+import org.apache.flink.table.runtime.operators.join.stream.state.OuterJoinRecordStateViews;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * Streaming unbounded Join operator which supports SEMI/ANTI JOIN.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinInputSideSpec.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinInputSideSpec.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinInputSideSpec.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinInputSideSpec.java
index dc35538..f6beb02 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinInputSideSpec.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinInputSideSpec.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream.state;
+package org.apache.flink.table.runtime.operators.join.stream.state;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 import javax.annotation.Nullable;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateView.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateView.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateView.java
index 58c4d84..d242542 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateView.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateView.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream.state;
+package org.apache.flink.table.runtime.operators.join.stream.state;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateViews.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateViews.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateViews.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateViews.java
index e3908fc..c2201f2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/JoinRecordStateViews.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/JoinRecordStateViews.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream.state;
+package org.apache.flink.table.runtime.operators.join.stream.state;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.MapState;
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.time.Time;
 import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.IterableIterator;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateView.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateView.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateView.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateView.java
index 32942a6..da089c3 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateView.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateView.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream.state;
+package org.apache.flink.table.runtime.operators.join.stream.state;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.table.dataformat.BaseRow;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateViews.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateViews.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateViews.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateViews.java
index c463992..2a146ae 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/stream/state/OuterJoinRecordStateViews.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/stream/state/OuterJoinRecordStateViews.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.stream.state;
+package org.apache.flink.table.runtime.operators.join.stream.state;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.MapState;
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.IterableIterator;
 
 import java.util.ArrayList;
@@ -37,7 +37,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.flink.table.runtime.join.stream.state.JoinRecordStateViews.createTtlConfig;
+import static org.apache.flink.table.runtime.operators.join.stream.state.JoinRecordStateViews.createTtlConfig;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
index dd88950..294db45 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/BaseTwoInputStreamOperatorWithStateRetention.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.temporal;
+package org.apache.flink.table.runtime.operators.join.temporal;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.state.ValueState;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalProcessTimeJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalProcessTimeJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java
index c67b480..13ce348 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalProcessTimeJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.temporal;
+package org.apache.flink.table.runtime.operators.join.temporal;
 
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.api.common.state.ValueState;
@@ -30,9 +30,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 /**
  * The operator to temporal join a stream on processing time.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalRowTimeJoinOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalRowTimeJoinOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java
index 56d6e10..64be99b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/join/temporal/TemporalRowTimeJoinOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join.temporal;
+package org.apache.flink.table.runtime.operators.join.temporal;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -33,9 +33,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 import java.io.IOException;
 import java.io.Serializable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/BaseRowEventComparator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/BaseRowEventComparator.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/BaseRowEventComparator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/BaseRowEventComparator.java
index e6472e0..903cd56 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/BaseRowEventComparator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/BaseRowEventComparator.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.match;
+package org.apache.flink.table.runtime.operators.match;
 
 import org.apache.flink.cep.EventComparator;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 
 /**
  * An implementation of {@link EventComparator} based on a generated {@link RecordComparator}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/IterativeConditionRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/IterativeConditionRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java
index 84adde1..e92c94c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/IterativeConditionRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.match;
+package org.apache.flink.table.runtime.operators.match;
 
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.cep.pattern.conditions.RichIterativeCondition;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
 
 /**
  * A {@link RichIterativeCondition} wrapper to delegate invocation to the code generated
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/PatternProcessFunctionRunner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/PatternProcessFunctionRunner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java
index e95f1de..3415fe0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/PatternProcessFunctionRunner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.match;
+package org.apache.flink.table.runtime.operators.match;
 
 import org.apache.flink.api.common.functions.util.FunctionUtils;
 import org.apache.flink.cep.functions.PatternProcessFunction;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
 import org.apache.flink.util.Collector;
 
 import java.util.List;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/RowtimeProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/RowtimeProcessFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java
index 0c7c86d..229db21 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/match/RowtimeProcessFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/match/RowtimeProcessFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.match;
+package org.apache.flink.table.runtime.operators.match;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/AbstractRowTimeUnboundedPrecedingOver.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/AbstractRowTimeUnboundedPrecedingOver.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java
index 922874a..e5a3216 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/AbstractRowTimeUnboundedPrecedingOver.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -28,12 +28,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/BufferDataOverWindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperator.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/BufferDataOverWindowOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperator.java
index e1d5b6b..73f955d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/BufferDataOverWindowOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.streaming.api.operators.BoundedOneInput;
@@ -25,14 +25,14 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
 import org.apache.flink.table.runtime.context.ExecutionContextImpl;
-import org.apache.flink.table.runtime.over.frame.OverWindowFrame;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.operators.over.frame.OverWindowFrame;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
 
 /**
  * the operator for OVER window need cache data by ResettableExternalBuffer for {@link OverWindowFrame}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/NonBufferOverWindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/NonBufferOverWindowOperator.java
similarity index 87%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/NonBufferOverWindowOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/NonBufferOverWindowOperator.java
index 9d77ac4..23ab9b6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/NonBufferOverWindowOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/NonBufferOverWindowOperator.java
@@ -16,20 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
 
 /**
  * The operator for OVER window don't need cache data.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRangeBoundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java
index c42018e..a344c35 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRangeBoundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -29,12 +29,12 @@ import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRowsBoundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRowsBoundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java
index fea641d..5e9f5bc 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeRowsBoundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -29,12 +29,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeUnboundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeUnboundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java
index 67eb982..e8545f6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/ProcTimeUnboundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
@@ -24,12 +24,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeBoundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java
index b30416e..7e10a8a 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeBoundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -28,12 +28,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeUnboundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeUnboundedPrecedingFunction.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeUnboundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeUnboundedPrecedingFunction.java
index bf04c65..6e261a0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRangeUnboundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeUnboundedPrecedingFunction.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsBoundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsBoundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java
index 9572973..f49a344 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsBoundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -28,12 +28,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsUnboundedPrecedingFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsUnboundedPrecedingFunction.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsUnboundedPrecedingFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsUnboundedPrecedingFunction.java
index 9fd46d9..845d049 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/RowTimeRowsUnboundedPrecedingFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsUnboundedPrecedingFunction.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over;
+package org.apache.flink.table.runtime.operators.over;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/InsensitiveOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/InsensitiveOverFrame.java
similarity index 88%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/InsensitiveOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/InsensitiveOverFrame.java
index 4c795ac..d02f294 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/InsensitiveOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/InsensitiveOverFrame.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OffsetOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OffsetOverFrame.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OffsetOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OffsetOverFrame.java
index 7f36bfc..804b62f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OffsetOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OffsetOverFrame.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 
 import java.io.Serializable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OverWindowFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OverWindowFrame.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OverWindowFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OverWindowFrame.java
index 7870e20..3fc3da6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/OverWindowFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/OverWindowFrame.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeSlidingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeSlidingOverFrame.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeSlidingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeSlidingOverFrame.java
index 773aee4..df529d2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeSlidingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeSlidingOverFrame.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 import org.apache.flink.table.types.logical.RowType;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedFollowingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedFollowingOverFrame.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedFollowingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedFollowingOverFrame.java
index f230f5d..23d4771 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedFollowingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedFollowingOverFrame.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedPrecedingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedPrecedingOverFrame.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedPrecedingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedPrecedingOverFrame.java
index 65da470..3cd3184 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RangeUnboundedPrecedingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RangeUnboundedPrecedingOverFrame.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 
 /**
  * The range UnboundPreceding window frame calculates frames with the following SQL form:
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowSlidingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowSlidingOverFrame.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowSlidingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowSlidingOverFrame.java
index cbe9d11..c387609 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowSlidingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowSlidingOverFrame.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedFollowingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedFollowingOverFrame.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedFollowingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedFollowingOverFrame.java
index 718204b..97201e3 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedFollowingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedFollowingOverFrame.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedPrecedingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedPrecedingOverFrame.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedPrecedingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedPrecedingOverFrame.java
index 145f3c9..f6a1ade 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/RowUnboundedPrecedingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/RowUnboundedPrecedingOverFrame.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/SlidingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/SlidingOverFrame.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/SlidingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/SlidingOverFrame.java
index 6baf2ec..02cf8a9 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/SlidingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/SlidingOverFrame.java
@@ -16,18 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 import java.util.ArrayDeque;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedFollowingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedFollowingOverFrame.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedFollowingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedFollowingOverFrame.java
index ec9e62a..4878408 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedFollowingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedFollowingOverFrame.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 /**
  * The UnboundedFollowing window frame.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedOverWindowFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedOverWindowFrame.java
similarity index 88%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedOverWindowFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedOverWindowFrame.java
index 7b189cf..76f2aa5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedOverWindowFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedOverWindowFrame.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 /**
  * The unbounded window frame calculates frames with the following SQL forms:
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedPrecedingOverFrame.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedPrecedingOverFrame.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedPrecedingOverFrame.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedPrecedingOverFrame.java
index 0add1dc..43c8f2c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/over/frame/UnboundedPrecedingOverFrame.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/over/frame/UnboundedPrecedingOverFrame.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.over.frame;
+package org.apache.flink.table.runtime.operators.over.frame;
 
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.dataview.PerKeyStateDataViewStore;
-import org.apache.flink.table.generated.AggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.context.ExecutionContext;
+import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore;
+import org.apache.flink.table.runtime.generated.AggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.util.ResettableExternalBuffer;
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AbstractTopNFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AbstractTopNFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java
index dc9b278..f18a22c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AbstractTopNFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
@@ -30,11 +30,11 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.functions.KeyedProcessFunctionWithCleanupState;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AppendOnlyTopNFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AppendOnlyTopNFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java
index 0348c47..11a99b2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/AppendOnlyTopNFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.MapState;
@@ -25,10 +25,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.ListTypeInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.LRUMap;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRange.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRange.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRange.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRange.java
index b4bb201..97d6299 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRange.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRange.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import java.util.List;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRangeWithoutEnd.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRangeWithoutEnd.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRangeWithoutEnd.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRangeWithoutEnd.java
index 0d0ddd3..8db9285 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/ConstantRankRangeWithoutEnd.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/ConstantRankRangeWithoutEnd.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import java.util.List;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankRange.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankRange.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankRange.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankRange.java
index e14b4bf..99ec360 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankRange.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankRange.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import java.io.Serializable;
 import java.util.List;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankType.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankType.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankType.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankType.java
index d6573dd..d763df4 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RankType.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RankType.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 /**
  * An enumeration of rank type, usable to show how exactly generate rank number.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RetractableTopNFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RetractableTopNFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java
index 20e77ec..11820e0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/RetractableTopNFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -27,12 +27,12 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.SortedMapTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/TopNBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/TopNBuffer.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/TopNBuffer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/TopNBuffer.java
index 794c4f9..ac82128 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/TopNBuffer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/TopNBuffer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import org.apache.flink.table.dataformat.BaseRow;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/UpdatableTopNFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/UpdatableTopNFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java
index 708b0c0..68a3c61 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/UpdatableTopNFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.MapState;
@@ -31,10 +31,10 @@ import org.apache.flink.runtime.state.FunctionInitializationContext;
 import org.apache.flink.runtime.state.FunctionSnapshotContext;
 import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.keyselector.BaseRowKeySelector;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.LRUMap;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/VariableRankRange.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/VariableRankRange.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/VariableRankRange.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/VariableRankRange.java
index a065fe3..a383d38 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/rank/VariableRankRange.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/VariableRankRange.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.rank;
+package org.apache.flink.table.runtime.operators.rank;
 
 import java.util.List;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/AbstractBinaryExternalMerger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/AbstractBinaryExternalMerger.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/AbstractBinaryExternalMerger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/AbstractBinaryExternalMerger.java
index d559edd..6db5cce 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/AbstractBinaryExternalMerger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/AbstractBinaryExternalMerger.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView;
 import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelWriterOutputView;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BaseTemporalSortOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BaseTemporalSortOperator.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BaseTemporalSortOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BaseTemporalSortOperator.java
index 3fdd193..208fc1b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BaseTemporalSortOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BaseTemporalSortOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalMerger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalMerger.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalMerger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalMerger.java
index 97a9b75..e10ee70 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalMerger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalMerger.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.runtime.io.disk.ChannelReaderInputViewIterator;
 import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
index d752fdb..ef81ddb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryExternalSorter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.configuration.AlgorithmOptions;
@@ -35,13 +35,13 @@ import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 import org.apache.flink.table.runtime.io.ChannelWithMeta;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryInMemorySortBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryInMemorySortBuffer.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryInMemorySortBuffer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryInMemorySortBuffer.java
index f51cc1a..8a6e4f0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryInMemorySortBuffer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryInMemorySortBuffer.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.EOFException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryIndexedSortable.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryIndexedSortable.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryIndexedSortable.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryIndexedSortable.java
index 76cf216..87b0c3c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryIndexedSortable.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryIndexedSortable.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.RandomAccessInputView;
@@ -24,10 +24,10 @@ import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.runtime.operators.sort.IndexedSortable;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVExternalMerger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVExternalMerger.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVExternalMerger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVExternalMerger.java
index 137dcdd..5177e92 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVExternalMerger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVExternalMerger.java
@@ -15,16 +15,16 @@
 * limitations under the License.
 */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.io.disk.iomanager.AbstractChannelReaderInputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVInMemorySortBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVInMemorySortBuffer.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVInMemorySortBuffer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVInMemorySortBuffer.java
index eb5a3c6..127b4dd 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryKVInMemorySortBuffer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryKVInMemorySortBuffer.java
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.RandomAccessInputView;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryMergeIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryMergeIterator.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryMergeIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryMergeIterator.java
index 9e447aa..1fd2973 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryMergeIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BinaryMergeIterator.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.runtime.operators.sort.MergeIterator;
 import org.apache.flink.runtime.operators.sort.PartialOrderPriorityQueue;
-import org.apache.flink.table.generated.RecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
index b8e7a69..ca969ca 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/BufferedKVExternalSorter.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
@@ -27,13 +27,13 @@ import org.apache.flink.runtime.operators.sort.IndexedSorter;
 import org.apache.flink.runtime.operators.sort.QuickSort;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
 import org.apache.flink.table.runtime.io.ChannelWithMeta;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ChannelReaderKVInputViewIterator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ChannelReaderKVInputViewIterator.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ChannelReaderKVInputViewIterator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ChannelReaderKVInputViewIterator.java
index 4b05756..2b4dcc4 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ChannelReaderKVInputViewIterator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ChannelReaderKVInputViewIterator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/LimitOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/LimitOperator.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/LimitOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/LimitOperator.java
index 93c7743..618b154 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/LimitOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/LimitOperator.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ListMemorySegmentPool.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ListMemorySegmentPool.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ListMemorySegmentPool.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ListMemorySegmentPool.java
index ef4a128..c511db8 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ListMemorySegmentPool.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ListMemorySegmentPool.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ProcTimeSortOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ProcTimeSortOperator.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ProcTimeSortOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ProcTimeSortOperator.java
index d623c95..c162997 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/ProcTimeSortOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/ProcTimeSortOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
@@ -24,9 +24,9 @@ import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.streaming.api.operators.InternalTimer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RankOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RankOperator.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RankOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RankOperator.java
index d43497e..f4ec7ca 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RankOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RankOperator.java
@@ -16,18 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
 
 /**
  * Rank operator to compute top N.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RowTimeSortOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RowTimeSortOperator.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RowTimeSortOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RowTimeSortOperator.java
index 3257406..f4c67c9 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/RowTimeSortOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/RowTimeSortOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
@@ -28,9 +28,9 @@ import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.streaming.api.operators.InternalTimer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortLimitOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortLimitOperator.java
similarity index 91%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortLimitOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortLimitOperator.java
index 6a4c270..f8ac530 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortLimitOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortLimitOperator.java
@@ -16,17 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.streaming.api.operators.BoundedOneInput;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
 import org.apache.flink.util.Collector;
 
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortOperator.java
similarity index 87%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortOperator.java
index d98f6c0..bace3b7 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.streaming.api.operators.BoundedOneInput;
@@ -24,14 +24,14 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedNormalizedKeyComputer;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortUtil.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java
index 9800832..0494f2e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SortUtil.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SortUtil.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.common.typeutils.base.NormalizedKeyUtil;
 import org.apache.flink.core.memory.MemorySegment;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SpillChannelManager.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SpillChannelManager.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SpillChannelManager.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SpillChannelManager.java
index 06adfca..a3106a9 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/SpillChannelManager.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/SpillChannelManager.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/StreamSortOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/StreamSortOperator.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/StreamSortOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/StreamSortOperator.java
index 3ce0d0f..b044022 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/StreamSortOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/sort/StreamSortOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.sort;
+package org.apache.flink.table.runtime.operators.sort;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ListState;
@@ -32,12 +32,12 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.TableStreamOperator;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.StreamRecordCollector;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/values/ValuesInputFormat.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/values/ValuesInputFormat.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/values/ValuesInputFormat.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/values/ValuesInputFormat.java
index e145a9f..3e61298 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/values/ValuesInputFormat.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/values/ValuesInputFormat.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.values;
+package org.apache.flink.table.runtime.operators.values;
 
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.common.io.NonParallelInput;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.core.io.GenericInputSplit;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.GeneratedInput;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.generated.GeneratedInput;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/CountWindow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/CountWindow.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/CountWindow.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/CountWindow.java
index 3e0c240..813ceb6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/CountWindow.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/CountWindow.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.api.window;
+package org.apache.flink.table.runtime.operators.window;
 
 import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/TimeWindow.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/TimeWindow.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java
index 5d93f83..9582190 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/TimeWindow.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/TimeWindow.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.api.window;
+package org.apache.flink.table.runtime.operators.window;
 
 import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/Window.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/Window.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/Window.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/Window.java
index c5d99d2..2156194 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/window/Window.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/Window.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.api.window;
+package org.apache.flink.table.runtime.operators.window;
 
 
 /**
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java
index 9d383b6..df4c191 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window;
+package org.apache.flink.table.runtime.operators.window;
 
 import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.State;
@@ -40,25 +40,24 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TimestampedCollector;
 import org.apache.flink.streaming.api.operators.Triggerable;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BaseRowUtil;
-import org.apache.flink.table.dataview.PerWindowStateDataViewStore;
-import org.apache.flink.table.generated.GeneratedNamespaceAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.generated.RecordEqualiser;
-import org.apache.flink.table.runtime.window.assigners.MergingWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.PanedWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.WindowAssigner;
-import org.apache.flink.table.runtime.window.internal.GeneralWindowProcessFunction;
-import org.apache.flink.table.runtime.window.internal.InternalWindowProcessFunction;
-import org.apache.flink.table.runtime.window.internal.MergingWindowProcessFunction;
-import org.apache.flink.table.runtime.window.internal.PanedWindowProcessFunction;
-import org.apache.flink.table.runtime.window.triggers.Trigger;
+import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore;
+import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.operators.window.assigners.MergingWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.PanedWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
+import org.apache.flink.table.runtime.operators.window.internal.GeneralWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.internal.InternalWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.internal.MergingWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.internal.PanedWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.triggers.Trigger;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -81,7 +80,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  *
  * <p>Each pane gets its own instance of the provided {@code Trigger}. This trigger determines when
  * the contents of the pane should be processed to emit results. When a trigger fires,
- * the given {@link org.apache.flink.table.generated.NamespaceAggsHandleFunction}
+ * the given {@link org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction}
  * is invoked to produce the results that are emitted for the pane to which the {@code Trigger}
  * belongs.
  *
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperatorBuilder.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java
similarity index 84%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperatorBuilder.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java
index c381386..3e20933 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/WindowOperatorBuilder.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java
@@ -16,25 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window;
+package org.apache.flink.table.runtime.operators.window;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.java.ClosureCleaner;
-import org.apache.flink.table.generated.GeneratedNamespaceAggsHandleFunction;
-import org.apache.flink.table.generated.GeneratedRecordEqualiser;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.generated.RecordEqualiser;
-import org.apache.flink.table.runtime.window.assigners.CountSlidingWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.CountTumblingWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.InternalTimeWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.SessionWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.SlidingWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.TumblingWindowAssigner;
-import org.apache.flink.table.runtime.window.assigners.WindowAssigner;
-import org.apache.flink.table.runtime.window.triggers.ElementTriggers;
-import org.apache.flink.table.runtime.window.triggers.EventTimeTriggers;
-import org.apache.flink.table.runtime.window.triggers.ProcessingTimeTriggers;
-import org.apache.flink.table.runtime.window.triggers.Trigger;
+import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.generated.RecordEqualiser;
+import org.apache.flink.table.runtime.operators.window.assigners.CountSlidingWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.CountTumblingWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.InternalTimeWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.SessionWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.SlidingWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.TumblingWindowAssigner;
+import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
+import org.apache.flink.table.runtime.operators.window.triggers.ElementTriggers;
+import org.apache.flink.table.runtime.operators.window.triggers.EventTimeTriggers;
+import org.apache.flink.table.runtime.operators.window.triggers.ProcessingTimeTriggers;
+import org.apache.flink.table.runtime.operators.window.triggers.Trigger;
 import org.apache.flink.table.types.logical.LogicalType;
 
 import java.time.Duration;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountSlidingWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountSlidingWindowAssigner.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountSlidingWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountSlidingWindowAssigner.java
index 3ab3f5a..286729b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountSlidingWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountSlidingWindowAssigner.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.CountWindow;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.window.internal.InternalWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.CountWindow;
+import org.apache.flink.table.runtime.operators.window.internal.InternalWindowProcessFunction;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountTumblingWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountTumblingWindowAssigner.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountTumblingWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountTumblingWindowAssigner.java
index 689f2d5..182d5ff 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/CountTumblingWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/CountTumblingWindowAssigner.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.CountWindow;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.window.internal.InternalWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.CountWindow;
+import org.apache.flink.table.runtime.operators.window.internal.InternalWindowProcessFunction;
 
 import java.io.IOException;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/InternalTimeWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/InternalTimeWindowAssigner.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/InternalTimeWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/InternalTimeWindowAssigner.java
index af316a9..b1b32b1 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/InternalTimeWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/InternalTimeWindowAssigner.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 /**
  * The internal time window assigner which has some useful methods.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/MergingWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/MergingWindowAssigner.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/MergingWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/MergingWindowAssigner.java
index 8b51f16..f7b6bae 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/MergingWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/MergingWindowAssigner.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
 
 import java.util.Collection;
 import java.util.NavigableSet;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/PanedWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/PanedWindowAssigner.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/PanedWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/PanedWindowAssigner.java
index e510a90..eb813ad 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/PanedWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/PanedWindowAssigner.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
 
 /**
  * A {@code WindowAssigner} that window can be split into panes.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SessionWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SessionWindowAssigner.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SessionWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SessionWindowAssigner.java
index 7c10639..ddff56d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SessionWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SessionWindowAssigner.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.TimeWindow;
 import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
 
 import java.time.Duration;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SlidingWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SlidingWindowAssigner.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SlidingWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SlidingWindowAssigner.java
index aa3adaa..4a55fb0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/SlidingWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/SlidingWindowAssigner.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.TimeWindow;
 import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
 import org.apache.flink.util.IterableIterator;
 import org.apache.flink.util.MathUtils;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/TumblingWindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/TumblingWindowAssigner.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/TumblingWindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/TumblingWindowAssigner.java
index de3494c..124b9e4 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/TumblingWindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/TumblingWindowAssigner.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.TimeWindow;
 import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
 
 import java.time.Duration;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/WindowAssigner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/WindowAssigner.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/WindowAssigner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/WindowAssigner.java
index 173ae91..db06150 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/assigners/WindowAssigner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/assigners/WindowAssigner.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.assigners;
+package org.apache.flink.table.runtime.operators.window.assigners;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.window.internal.InternalWindowProcessFunction;
-import org.apache.flink.table.runtime.window.triggers.Trigger;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.internal.InternalWindowProcessFunction;
+import org.apache.flink.table.runtime.operators.window.triggers.Trigger;
 
 import java.io.IOException;
 import java.io.Serializable;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/HeapWindowsGrouping.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/HeapWindowsGrouping.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/HeapWindowsGrouping.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/HeapWindowsGrouping.java
index 8f2513d..4365862 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/HeapWindowsGrouping.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/HeapWindowsGrouping.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.grouping;
+package org.apache.flink.table.runtime.operators.window.grouping;
 
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.runtime.util.RowIterator;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/WindowsGrouping.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/WindowsGrouping.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/WindowsGrouping.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/WindowsGrouping.java
index f632a87..dd53f41 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/grouping/WindowsGrouping.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/grouping/WindowsGrouping.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.grouping;
+package org.apache.flink.table.runtime.operators.window.grouping;
 
-import org.apache.flink.table.api.window.TimeWindow;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
+import org.apache.flink.table.runtime.operators.window.TimeWindow;
 import org.apache.flink.table.runtime.util.RowIterator;
 import org.apache.flink.util.Preconditions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/GeneralWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/GeneralWindowProcessFunction.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/GeneralWindowProcessFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/GeneralWindowProcessFunction.java
index cedbcc6..6e44ae6 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/GeneralWindowProcessFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/GeneralWindowProcessFunction.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.internal;
+package org.apache.flink.table.runtime.operators.window.internal;
 
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.runtime.window.assigners.WindowAssigner;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
 
 import java.util.ArrayList;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/InternalWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/InternalWindowProcessFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java
index 5b2b6db..126a295 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/InternalWindowProcessFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.internal;
+package org.apache.flink.table.runtime.operators.window.internal;
 
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.runtime.window.assigners.WindowAssigner;
-import org.apache.flink.table.runtime.window.triggers.Trigger;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
+import org.apache.flink.table.runtime.operators.window.triggers.Trigger;
 
 import java.io.Serializable;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowProcessFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java
index 5bd66c7..c1a64a0 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowProcessFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.internal;
+package org.apache.flink.table.runtime.operators.window.internal;
 
 import org.apache.flink.api.common.state.MapState;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.runtime.window.assigners.MergingWindowAssigner;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.MergingWindowAssigner;
 
 import java.util.ArrayList;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowSet.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowSet.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowSet.java
index 1721a82..915f2e8 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/MergingWindowSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowSet.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.internal;
+package org.apache.flink.table.runtime.operators.window.internal;
 
 import org.apache.flink.api.common.state.MapState;
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.MergingWindowAssigner;
 import org.apache.flink.table.runtime.util.LRUMap;
-import org.apache.flink.table.runtime.window.assigners.MergingWindowAssigner;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/PanedWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/PanedWindowProcessFunction.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/PanedWindowProcessFunction.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/PanedWindowProcessFunction.java
index f87b4a8..e701512 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/internal/PanedWindowProcessFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/PanedWindowProcessFunction.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.internal;
+package org.apache.flink.table.runtime.operators.window.internal;
 
-import org.apache.flink.table.api.window.Window;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.generated.NamespaceAggsHandleFunction;
-import org.apache.flink.table.runtime.window.assigners.PanedWindowAssigner;
+import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.PanedWindowAssigner;
 
 import java.util.ArrayList;
 import java.util.Collection;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ElementTriggers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ElementTriggers.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ElementTriggers.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ElementTriggers.java
index ea8be4c..97d16a2 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ElementTriggers.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ElementTriggers.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.triggers;
+package org.apache.flink.table.runtime.operators.window.triggers;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
 
 /**
  * A {@link Trigger} that fires at some point after a specified number of
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/EventTimeTriggers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/EventTimeTriggers.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java
index 325dac4..fc09673 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/EventTimeTriggers.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.triggers;
+package org.apache.flink.table.runtime.operators.window.triggers;
 
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ProcessingTimeTriggers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ProcessingTimeTriggers.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java
index b703f53..cfae3f7 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/ProcessingTimeTriggers.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.triggers;
+package org.apache.flink.table.runtime.operators.window.triggers;
 
 import org.apache.flink.api.common.functions.ReduceFunction;
 import org.apache.flink.api.common.state.ReducingState;
 import org.apache.flink.api.common.state.ReducingStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.table.api.window.Window;
+import org.apache.flink.table.runtime.operators.window.Window;
 
 import java.time.Duration;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/Trigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/Trigger.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java
index 63a69b0..ca06abc 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/window/triggers/Trigger.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.window.triggers;
+package org.apache.flink.table.runtime.operators.window.triggers;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.common.state.MergingState;
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.metrics.MetricGroup;
-import org.apache.flink.table.api.window.Window;
-import org.apache.flink.table.runtime.window.assigners.WindowAssigner;
+import org.apache.flink.table.runtime.operators.window.Window;
+import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
 
 import java.io.Serializable;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchAssignerOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchAssignerOperator.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchAssignerOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchAssignerOperator.java
index 8d2e5e2..f8bc196 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchAssignerOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchAssignerOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.watermarkassigner;
+package org.apache.flink.table.runtime.operators.wmassigners;
 
 import org.apache.flink.metrics.Gauge;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchedWatermarkAssignerOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchedWatermarkAssignerOperator.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchedWatermarkAssignerOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchedWatermarkAssignerOperator.java
index 2411a0b..a803144 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/MiniBatchedWatermarkAssignerOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/MiniBatchedWatermarkAssignerOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.watermarkassigner;
+package org.apache.flink.table.runtime.operators.wmassigners;
 
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.ChainingStrategy;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperator.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java
index 814810f..049790f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/watermarkassigner/WatermarkAssignerOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.watermarkassigner;
+package org.apache.flink.table.runtime.operators.wmassigners;
 
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/partitioner/BinaryHashPartitioner.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/partitioner/BinaryHashPartitioner.java
index 198047a..4bdfc1c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/partitioner/BinaryHashPartitioner.java
@@ -16,15 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime;
+package org.apache.flink.table.runtime.partitioner;
 
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedHashFunction;
-import org.apache.flink.table.generated.HashFunction;
+import org.apache.flink.table.runtime.generated.GeneratedHashFunction;
+import org.apache.flink.table.runtime.generated.HashFunction;
 import org.apache.flink.util.MathUtils;
 
 import java.util.Arrays;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassDataTypeConverter.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassDataTypeConverter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassDataTypeConverter.java
index c6e6cfe..350738d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassDataTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassDataTypeConverter.java
@@ -16,10 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LegacyTypeInformationType;
 import org.apache.flink.table.types.utils.TypeConversions;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassLogicalTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassLogicalTypeConverter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java
index eaaf405..7c7ad74 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/ClassLogicalTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/ClassLogicalTypeConverter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.table.dataformat.BaseArray;
 import org.apache.flink.table.dataformat.BaseMap;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/InternalSerializers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java
similarity index 88%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/InternalSerializers.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java
index c8d5b76..928215e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/InternalSerializers.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/InternalSerializers.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -28,6 +28,12 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.api.common.typeutils.base.ShortSerializer;
 import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryGenericSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer;
+import org.apache.flink.table.runtime.typeutils.DecimalSerializer;
 import org.apache.flink.table.types.logical.ArrayType;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.IntType;
@@ -36,12 +42,6 @@ import org.apache.flink.table.types.logical.MapType;
 import org.apache.flink.table.types.logical.MultisetType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.typeutils.BinaryGenericSerializer;
-import org.apache.flink.table.typeutils.BinaryStringSerializer;
-import org.apache.flink.table.typeutils.DecimalSerializer;
 
 /**
  * {@link TypeSerializer} of {@link LogicalType} for internal sql engine execution data formats.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/LogicalTypeDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/LogicalTypeDataTypeConverter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java
index a61c420..2df059f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/LogicalTypeDataTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/LogicalTypeDataTypeConverter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -24,6 +24,10 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompositeType;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.dataformat.Decimal;
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.ArrayType;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LegacyTypeInformationType;
@@ -34,15 +38,12 @@ import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.TypeInformationAnyType;
 import org.apache.flink.table.types.logical.utils.LogicalTypeDefaultVisitor;
 import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
+import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
 
 /**
  * Converter between {@link DataType} and {@link LogicalType}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/PlannerTypeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/PlannerTypeUtils.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
index 08622aa..bdb1134 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/PlannerTypeUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java
@@ -16,8 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeRoot;
 import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoDataTypeConverter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
index ecdbaa3..044356c 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoDataTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
@@ -32,22 +32,26 @@ import org.apache.flink.table.dataformat.Decimal;
 import org.apache.flink.table.dataview.MapViewTypeInfo;
 import org.apache.flink.table.functions.AggregateFunctionDefinition;
 import org.apache.flink.table.functions.TableFunctionDefinition;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
 import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;
 
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
-import static org.apache.flink.table.types.PlannerTypeUtils.isPrimitive;
+import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
+import static org.apache.flink.table.runtime.types.PlannerTypeUtils.isPrimitive;
 
 /**
  * Converter between {@link TypeInformation} and {@link DataType}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoLogicalTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoLogicalTypeConverter.java
similarity index 89%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoLogicalTypeConverter.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoLogicalTypeConverter.java
index 67b281a..2c77199 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/types/TypeInfoLogicalTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoLogicalTypeConverter.java
@@ -16,16 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.types;
+package org.apache.flink.table.runtime.types;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.table.typeutils.BigDecimalTypeInfo;
 
-import static org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType;
+import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType;
 
 /**
  * Converter between {@link TypeInformation} and {@link LogicalType}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapSerializer.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapSerializer.java
index 344fb97..4f88e93 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.DataInputView;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapTypeInfo.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapTypeInfo.java
index 2321d28..abf5313 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractMapTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractMapTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.util.Preconditions;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractRowSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractRowSerializer.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractRowSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractRowSerializer.java
index 5043eb4..dd004d3 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/AbstractRowSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/AbstractRowSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.memory.AbstractPagedInputView;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseArraySerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseArraySerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java
index c749dfa..c2c50fb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseArraySerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseArraySerializer.java
@@ -1,12 +1,13 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.	See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.	You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *		http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -33,16 +34,16 @@ import org.apache.flink.table.dataformat.BinaryArrayWriter;
 import org.apache.flink.table.dataformat.BinaryWriter;
 import org.apache.flink.table.dataformat.GenericArray;
 import org.apache.flink.table.dataformat.TypeGetterSetters;
-import org.apache.flink.table.types.InternalSerializers;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.SegmentsUtil;
 import org.apache.flink.util.InstantiationUtil;
 
 import java.io.IOException;
 import java.lang.reflect.Array;
 import java.util.Arrays;
 
-import static org.apache.flink.table.types.ClassLogicalTypeConverter.getInternalClassForType;
+import static org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getInternalClassForType;
 
 /**
  * Serializer for {@link BaseArray}.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseMapSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializer.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseMapSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializer.java
index 766c3b5..3c59dfc 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseMapSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseMapSerializer.java
@@ -1,12 +1,13 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.	See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.	You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *		http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
@@ -33,9 +34,9 @@ import org.apache.flink.table.dataformat.BinaryArrayWriter;
 import org.apache.flink.table.dataformat.BinaryMap;
 import org.apache.flink.table.dataformat.BinaryWriter;
 import org.apache.flink.table.dataformat.GenericMap;
-import org.apache.flink.table.types.InternalSerializers;
+import org.apache.flink.table.runtime.types.InternalSerializers;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.SegmentsUtil;
 import org.apache.flink.util.InstantiationUtil;
 
 import org.slf4j.Logger;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowSerializer.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowSerializer.java
index 4bf0546..d05630d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerUtil;
@@ -36,7 +36,7 @@ import org.apache.flink.table.dataformat.BinaryRowWriter;
 import org.apache.flink.table.dataformat.BinaryWriter;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.TypeGetterSetters;
-import org.apache.flink.table.types.InternalSerializers;
+import org.apache.flink.table.runtime.types.InternalSerializers;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.util.InstantiationUtil;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowTypeInfo.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowTypeInfo.java
index 10987ff..481835f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BaseRowTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BaseRowTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.operators.Keys;
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeutils.CompositeType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.java.typeutils.TupleTypeInfoBase;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter;
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BigDecimalTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BigDecimalTypeInfo.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BigDecimalTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BigDecimalTypeInfo.java
index 74ebccd..29942ef 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BigDecimalTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BigDecimalTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryGenericSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryGenericSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java
index 7ecc905..b0ca78f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryGenericSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryGenericSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -29,7 +29,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.table.dataformat.BinaryGeneric;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.util.InstantiationUtil;
 
 import java.io.IOException;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryRowSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryRowSerializer.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryRowSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryRowSerializer.java
index c187471..ce6f7e3 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryRowSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryRowSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
@@ -29,7 +29,7 @@ import org.apache.flink.core.memory.MemorySegmentWritable;
 import org.apache.flink.runtime.memory.AbstractPagedInputView;
 import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 import java.io.IOException;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringSerializer.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringSerializer.java
index 6976cbc..83e6e3f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.util.SegmentsUtil;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 
 import java.io.IOException;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfo.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfo.java
index f4a2a19..2feeb77 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfoFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfoFactory.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfoFactory.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfoFactory.java
index 5969d89..f00d74e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/BinaryStringTypeInfoFactory.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/BinaryStringTypeInfoFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalSerializer.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalSerializer.java
index 14c4304..0f16061 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfo.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfo.java
index e71c868..6787a5f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfoFactory.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfoFactory.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfoFactory.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfoFactory.java
index f150f9e..5a81085 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/DecimalTypeInfoFactory.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/DecimalTypeInfoFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeinfo.TypeInfoFactory;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializer.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializer.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializer.java
index 63f4f8f..269d64b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializerSnapshot.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializerSnapshot.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializerSnapshot.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializerSnapshot.java
index d7a1dcf..7db3883 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapSerializerSnapshot.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapSerializerSnapshot.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.api.common.typeutils.NestedSerializersSnapshotDelegate;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapTypeInfo.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapTypeInfo.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapTypeInfo.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapTypeInfo.java
index 0e3cef7..2b6d237 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/SortedMapTypeInfo.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/SortedMapTypeInfo.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/TypeCheckUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/TypeCheckUtils.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java
index d1e9a5e..01c7ecf 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/typeutils/TypeCheckUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.typeutils;
+package org.apache.flink.table.runtime.typeutils;
 
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.LogicalTypeFamily;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/MurmurHashUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/MurmurHashUtil.java
similarity index 90%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/MurmurHashUtil.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/MurmurHashUtil.java
index dbae3e8..fdfeb31 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/MurmurHashUtil.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/MurmurHashUtil.java
@@ -1,12 +1,13 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.	See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.	You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *		http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.util;
+package org.apache.flink.table.runtime.util;
 
 import org.apache.flink.core.memory.MemorySegment;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/ResettableExternalBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/ResettableExternalBuffer.java
index 4e03785..fb4a80b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/ResettableExternalBuffer.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/ResettableExternalBuffer.java
@@ -34,8 +34,8 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.runtime.io.BinaryRowChannelInputViewIterator;
 import org.apache.flink.table.runtime.io.ChannelWithMeta;
-import org.apache.flink.table.typeutils.AbstractRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
+import org.apache.flink.table.runtime.typeutils.AbstractRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.MutableObjectIterator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/SegmentsUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/SegmentsUtil.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/SegmentsUtil.java
rename to flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/SegmentsUtil.java
index 47475a4..82a3e4b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/util/SegmentsUtil.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/SegmentsUtil.java
@@ -1,12 +1,13 @@
 /*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.	See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.	You may obtain a copy of the License at
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
  *
- *		http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -15,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.util;
+package org.apache.flink.table.runtime.util;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.core.memory.DataOutputView;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StringUtf8Utils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StringUtf8Utils.java
index c3487c5..f67141b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StringUtf8Utils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StringUtf8Utils.java
@@ -22,8 +22,8 @@ import org.apache.flink.core.memory.MemorySegment;
 import java.io.UnsupportedEncodingException;
 import java.util.Arrays;
 
-import static org.apache.flink.table.util.SegmentsUtil.allocateReuseBytes;
-import static org.apache.flink.table.util.SegmentsUtil.allocateReuseChars;
+import static org.apache.flink.table.runtime.util.SegmentsUtil.allocateReuseBytes;
+import static org.apache.flink.table.runtime.util.SegmentsUtil.allocateReuseChars;
 
 /**
  * String utf-8 utils.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/DoubleHashSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/DoubleHashSet.java
index 0f1c056..3f1d13e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/DoubleHashSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/DoubleHashSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.table.runtime.util.collections;
 
-import org.apache.flink.table.util.MurmurHashUtil;
+import org.apache.flink.table.runtime.util.MurmurHashUtil;
 
 /**
  * Double hash set.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/FloatHashSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/FloatHashSet.java
index 8d7e26e..d139198 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/FloatHashSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/FloatHashSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.table.runtime.util.collections;
 
-import org.apache.flink.table.util.MurmurHashUtil;
+import org.apache.flink.table.runtime.util.MurmurHashUtil;
 
 /**
  * Float hash set.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/IntHashSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/IntHashSet.java
index 1159917..af1775d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/IntHashSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/IntHashSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.table.runtime.util.collections;
 
-import org.apache.flink.table.util.MurmurHashUtil;
+import org.apache.flink.table.runtime.util.MurmurHashUtil;
 
 /**
  * Int hash set.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/LongHashSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/LongHashSet.java
index 37a8cd0..162576b 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/LongHashSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/LongHashSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.table.runtime.util.collections;
 
-import org.apache.flink.table.util.MurmurHashUtil;
+import org.apache.flink.table.runtime.util.MurmurHashUtil;
 
 /**
  * Long hash set.
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/ShortHashSet.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/ShortHashSet.java
index 54e42f9..dd3860e 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/ShortHashSet.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/collections/ShortHashSet.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.table.runtime.util.collections;
 
-import org.apache.flink.table.util.MurmurHashUtil;
+import org.apache.flink.table.runtime.util.MurmurHashUtil;
 
 /**
  * Short hash set.
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java
index 8723405..a86e001 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BaseRowTest.java
@@ -19,11 +19,11 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 import org.junit.Before;
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java
index 5063646..f64907e 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryArrayTest.java
@@ -22,13 +22,13 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.util.SegmentsUtil;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.util.SegmentsUtil;
 
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java
index 6351e78..fb437c8 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryRowTest.java
@@ -34,13 +34,13 @@ import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.RandomAccessInputView;
 import org.apache.flink.runtime.io.disk.RandomAccessOutputView;
 import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.runtime.typeutils.BaseArraySerializer;
+import org.apache.flink.table.runtime.typeutils.BaseMapSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.BaseArraySerializer;
-import org.apache.flink.table.typeutils.BaseMapSerializer;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java
index 6060213..59c93ec 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/BinaryStringTest.java
@@ -19,7 +19,7 @@ package org.apache.flink.table.dataformat;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.table.runtime.sort.SortUtil;
+import org.apache.flink.table.runtime.operators.sort.SortUtil;
 import org.apache.flink.table.runtime.util.StringUtf8Utils;
 
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/DataFormatConvertersTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/DataFormatConvertersTest.java
index 684694a..1d42b5f 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/DataFormatConvertersTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/DataFormatConvertersTest.java
@@ -33,12 +33,12 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.table.dataformat.DataFormatConverters.DataFormatConverter;
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
+import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
+import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.VarCharType;
 import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.typeutils.DecimalTypeInfo;
 import org.apache.flink.types.Row;
 
 import org.junit.Assert;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java
index 78a6ba0..2ba0237 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/dataformat/NestedRowTest.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
 
 import org.junit.Test;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/CompileUtilsTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/CompileUtilsTest.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/CompileUtilsTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/CompileUtilsTest.java
index 2e67713..e5415c8 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/CompileUtilsTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/CompileUtilsTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.InvalidProgramException;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedCollectorWrapper.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedCollectorWrapper.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedCollectorWrapper.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedCollectorWrapper.java
index 2e3744c..b445a87 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedCollectorWrapper.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedCollectorWrapper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.util.Collector;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedFunctionWrapper.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedFunctionWrapper.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedFunctionWrapper.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedFunctionWrapper.java
index 934eca2..dddf2a8 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedFunctionWrapper.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedFunctionWrapper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.api.common.functions.Function;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedResultFutureWrapper.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedResultFutureWrapper.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedResultFutureWrapper.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedResultFutureWrapper.java
index ce7a096..74cbb93 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/generated/GeneratedResultFutureWrapper.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/generated/GeneratedResultFutureWrapper.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.generated;
+package org.apache.flink.table.runtime.generated;
 
 import org.apache.flink.streaming.api.functions.async.ResultFuture;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
index 1e4ebb0..ddc03f6 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
@@ -32,11 +32,11 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.runtime.join.HashJoinType;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.operators.join.HashJoinType;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.RowIterator;
 import org.apache.flink.table.runtime.util.UniformBinaryRowGenerator;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.util.MutableObjectIterator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
index 1933493..ea03cea 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
@@ -29,9 +29,9 @@ import org.apache.flink.runtime.operators.testutils.UnionIterator;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.runtime.util.RowIterator;
 import org.apache.flink.table.runtime.util.UniformBinaryRowGenerator;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.junit.Assert;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/BytesHashMapTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java
similarity index 99%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/BytesHashMapTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java
index 52e31b3..573422e 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/BytesHashMapTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/BytesHashMapTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
 import org.apache.flink.table.dataformat.BinaryString;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.BooleanType;
 import org.apache.flink.table.types.logical.DoubleType;
@@ -35,7 +36,6 @@ import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.logical.SmallIntType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.junit.Assert;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/HashAggTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/HashAggTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java
index 5a74016..abc0cdd 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/HashAggTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/HashAggTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/SumHashAggTestOperator.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/SumHashAggTestOperator.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/SumHashAggTestOperator.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/SumHashAggTestOperator.java
index 463db03..34dd895 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/aggregate/SumHashAggTestOperator.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/SumHashAggTestOperator.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.aggregate;
+package org.apache.flink.table.runtime.operators.aggregate;
 
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
@@ -31,13 +31,13 @@ import org.apache.flink.table.dataformat.BinaryRowWriter;
 import org.apache.flink.table.dataformat.GenericRow;
 import org.apache.flink.table.dataformat.JoinedRow;
 import org.apache.flink.table.dataformat.util.BinaryRowUtil;
-import org.apache.flink.table.runtime.sort.BufferedKVExternalSorter;
-import org.apache.flink.table.runtime.sort.IntNormalizedKeyComputer;
-import org.apache.flink.table.runtime.sort.IntRecordComparator;
+import org.apache.flink.table.runtime.operators.sort.BufferedKVExternalSorter;
+import org.apache.flink.table.runtime.operators.sort.IntNormalizedKeyComputer;
+import org.apache.flink.table.runtime.operators.sort.IntRecordComparator;
+import org.apache.flink.table.runtime.typeutils.BinaryRowSerializer;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.typeutils.BinaryRowSerializer;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/MapBundleOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperatorTest.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/MapBundleOperatorTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperatorTest.java
index b825b03..724fd82 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/MapBundleOperatorTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/MapBundleOperatorTest.java
@@ -16,13 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle;
+package org.apache.flink.table.runtime.operators.bundle;
 
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.table.runtime.bundle.trigger.CountBundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.trigger.CountBundleTrigger;
 import org.apache.flink.util.Collector;
 
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTriggerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTriggerTest.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTriggerTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTriggerTest.java
index 2b4ffd0..7d9bcc1 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountBundleTriggerTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountBundleTriggerTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.junit.Test;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTriggerTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTriggerTest.java
similarity index 96%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTriggerTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTriggerTest.java
index 8722253..e65f099 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/CountCoBundleTriggerTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/CountCoBundleTriggerTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 import org.junit.Test;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/TestTriggerCallback.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/TestTriggerCallback.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/TestTriggerCallback.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/TestTriggerCallback.java
index 1d89349..b56d5b3 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/bundle/trigger/TestTriggerCallback.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/bundle/trigger/TestTriggerCallback.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.bundle.trigger;
+package org.apache.flink.table.runtime.operators.bundle.trigger;
 
 /**
  * A bundle trigger callback which simply track the number of triggers.
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionTestBase.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionTestBase.java
similarity index 93%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionTestBase.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionTestBase.java
index f217e7c..1964ba4 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateFunctionTestBase.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionTestBase.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor;
 import org.apache.flink.table.runtime.util.BinaryRowKeySelector;
 import org.apache.flink.table.runtime.util.GenericRowRecordSortComparator;
 import org.apache.flink.table.types.logical.BigIntType;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 
 /**
  * Base class of tests for all kinds of DeduplicateFunction.
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunctionTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
index cbfc2be..0c0f1c7 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepFirstRowFunctionTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunctionTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunctionTest.java
similarity index 98%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunctionTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunctionTest.java
index c60b22c..a30134f 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/DeduplicateKeepLastRowFunctionTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateKeepLastRowFunctionTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
similarity index 92%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
index 3d79fff..80dab78 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepFirstRowFunctionTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -24,8 +24,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator;
-import org.apache.flink.table.runtime.bundle.trigger.CountBundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator;
+import org.apache.flink.table.runtime.operators.bundle.trigger.CountBundleTrigger;
 
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
similarity index 95%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
index 1779786..8482564 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/deduplicate/MiniBatchDeduplicateKeepLastRowFunctionTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.deduplicate;
+package org.apache.flink.table.runtime.operators.deduplicate;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -24,8 +24,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.table.dataformat.BaseRow;
-import org.apache.flink.table.runtime.bundle.KeyedMapBundleOperator;
-import org.apache.flink.table.runtime.bundle.trigger.CountBundleTrigger;
+import org.apache.flink.table.runtime.operators.bundle.KeyedMapBundleOperator;
+import org.apache.flink.table.runtime.operators.bundle.trigger.CountBundleTrigger;
 
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/AsyncLookupJoinHarnessTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java
similarity index 94%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/AsyncLookupJoinHarnessTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java
index 5b6827b..7c33e46 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/AsyncLookupJoinHarnessTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.AbstractRichFunction;
@@ -34,19 +34,19 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryString;
 import org.apache.flink.table.dataformat.GenericRow;
-import org.apache.flink.table.generated.GeneratedFunctionWrapper;
-import org.apache.flink.table.generated.GeneratedResultFutureWrapper;
 import org.apache.flink.table.runtime.collector.TableFunctionCollector;
 import org.apache.flink.table.runtime.collector.TableFunctionResultFuture;
-import org.apache.flink.table.runtime.join.lookup.AsyncLookupJoinRunner;
-import org.apache.flink.table.runtime.join.lookup.AsyncLookupJoinWithCalcRunner;
-import org.apache.flink.table.runtime.join.lookup.LookupJoinRunner;
-import org.apache.flink.table.runtime.join.lookup.LookupJoinWithCalcRunner;
+import org.apache.flink.table.runtime.generated.GeneratedFunctionWrapper;
+import org.apache.flink.table.runtime.generated.GeneratedResultFutureWrapper;
+import org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinRunner;
+import org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinWithCalcRunner;
+import org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner;
+import org.apache.flink.table.runtime.operators.join.lookup.LookupJoinWithCalcRunner;
+import org.apache.flink.table.runtime.typeutils.BaseRowSerializer;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.VarCharType;
-import org.apache.flink.table.typeutils.BaseRowSerializer;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.Collector;
 
 import org.junit.Test;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2HashJoinOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2HashJoinOperatorTest.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2HashJoinOperatorTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2HashJoinOperatorTest.java
index cfd8ae9..50d1017 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2HashJoinOperatorTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2HashJoinOperatorTest.java
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.api.common.functions.AbstractRichFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -32,14 +32,14 @@ import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
 import org.apache.flink.table.dataformat.JoinedRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.GeneratedProjection;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.generated.Projection;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.generated.Projection;
+import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
 import org.apache.flink.table.runtime.util.UniformBinaryRowGenerator;
 import org.apache.flink.table.types.logical.IntType;
 import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.typeutils.BaseRowTypeInfo;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.junit.Assert;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2SortMergeJoinOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java
similarity index 88%
rename from flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2SortMergeJoinOperatorTest.java
rename to flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java
index 994b553..34f9b59 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/join/Int2SortMergeJoinOperatorTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/join/Int2SortMergeJoinOperatorTest.java
@@ -7,7 +7,7 @@
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  *
- * http://www.apache.org/licenses/LICENSE-2.0
+ *     http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -16,24 +16,24 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.join;
+package org.apache.flink.table.runtime.operators.join;
 
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.table.dataformat.BinaryRow;
-import org.apache.flink.table.generated.GeneratedJoinCondition;
-import org.apache.flink.table.generated.GeneratedNormalizedKeyComputer;
-import org.apache.flink.table.generated.GeneratedProjection;
-import org.apache.flink.table.generated.GeneratedRecordComparator;
-import org.apache.flink.table.generated.JoinCondition;
-import org.apache.flink.table.generated.NormalizedKeyComputer;
-import org.apache.flink.table.generated.Projection;
-import org.apache.flink.table.generated.RecordComparator;
-import org.apache.flink.table.runtime.join.Int2HashJoinOperatorTest.MyProjection;
-import org.apache.flink.table.runtime.sort.IntNormalizedKeyComputer;
-import org.apache.flink.table.runtime.sort.IntRecordComparator;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.GeneratedProjection;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
... 1495 lines suppressed ...


[flink] 01/03: [FLINK-13266][table] Move OptimizerConfigOptions & ExecutionConfigOptions to table-api-java module

Posted by ja...@apache.org.
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 66e55d60f819c5a1f809830190461fb6ac341b0b
Author: godfreyhe <go...@163.com>
AuthorDate: Wed Jul 17 19:29:02 2019 +0800

    [FLINK-13266][table] Move OptimizerConfigOptions & ExecutionConfigOptions to table-api-java module
---
 .../flink/table/api/config}/ExecutionConfigOptions.java       | 11 ++++++-----
 .../flink/table/api/config}/OptimizerConfigOptions.java       |  4 +++-
 .../java/org/apache/flink/table/executor/BatchExecutor.java   |  2 +-
 .../flink/table/plan/nodes/resource/NodeResourceUtil.java     |  2 +-
 .../java/org/apache/flink/table/plan/util/OperatorType.java   |  2 +-
 .../org/apache/flink/table/util/AggregatePhaseStrategy.java   |  2 +-
 .../java/org/apache/flink/table/util/TableConfigUtils.java    |  4 ++--
 .../flink/table/codegen/NestedLoopJoinCodeGenerator.scala     |  2 +-
 .../flink/table/plan/nodes/common/CommonLookupJoin.scala      |  3 ++-
 .../flink/table/plan/nodes/logical/FlinkLogicalSort.scala     |  2 +-
 .../table/plan/nodes/physical/batch/BatchExecExchange.scala   |  2 +-
 .../nodes/physical/batch/BatchExecHashAggregateBase.scala     |  2 +-
 .../table/plan/nodes/physical/batch/BatchExecHashJoin.scala   |  2 +-
 .../physical/batch/BatchExecHashWindowAggregateBase.scala     |  2 +-
 .../plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala   |  2 +-
 .../plan/nodes/physical/batch/BatchExecOverAggregate.scala    |  3 ++-
 .../flink/table/plan/nodes/physical/batch/BatchExecSort.scala |  2 +-
 .../plan/nodes/physical/batch/BatchExecSortMergeJoin.scala    |  2 +-
 .../physical/batch/BatchExecSortWindowAggregateBase.scala     |  2 +-
 .../plan/nodes/physical/stream/StreamExecDeduplicate.scala    |  3 ++-
 .../physical/stream/StreamExecGlobalGroupAggregate.scala      |  3 ++-
 .../plan/nodes/physical/stream/StreamExecGroupAggregate.scala |  2 +-
 .../nodes/physical/stream/StreamExecWatermarkAssigner.scala   |  3 ++-
 .../plan/optimize/StreamCommonSubGraphBasedOptimizer.scala    |  3 ++-
 .../flink/table/plan/optimize/program/FlinkBatchProgram.scala |  2 +-
 .../table/plan/optimize/program/FlinkStreamProgram.scala      |  2 +-
 .../org/apache/flink/table/plan/reuse/SubplanReuser.scala     |  3 ++-
 .../rules/logical/PushFilterIntoTableSourceScanRule.scala     |  2 +-
 .../flink/table/plan/rules/logical/SplitAggregateRule.scala   |  3 ++-
 .../plan/rules/physical/batch/BatchExecHashAggRule.scala      |  2 +-
 .../plan/rules/physical/batch/BatchExecHashJoinRule.scala     |  3 ++-
 .../plan/rules/physical/batch/BatchExecSortAggRule.scala      |  2 +-
 .../rules/physical/batch/BatchExecWindowAggregateRule.scala   |  3 ++-
 .../rules/physical/stream/MiniBatchIntervalInferRule.scala    |  2 +-
 .../physical/stream/TwoStageOptimizedAggregateRule.scala      |  2 +-
 .../org/apache/flink/table/plan/util/AggregateUtil.scala      |  5 ++++-
 .../parallelism/ShuffleStageParallelismCalculatorTest.java    |  2 +-
 .../scala/org/apache/flink/table/api/batch/ExplainTest.scala  |  2 +-
 .../scala/org/apache/flink/table/api/stream/ExplainTest.scala |  2 +-
 .../org/apache/flink/table/catalog/CatalogTableITCase.scala   |  3 ++-
 .../flink/table/plan/batch/sql/DeadlockBreakupTest.scala      |  2 +-
 .../flink/table/plan/batch/sql/RemoveCollationTest.scala      |  3 ++-
 .../apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala |  3 ++-
 .../apache/flink/table/plan/batch/sql/SetOperatorsTest.scala  |  3 ++-
 .../org/apache/flink/table/plan/batch/sql/SortLimitTest.scala |  2 +-
 .../org/apache/flink/table/plan/batch/sql/SortTest.scala      |  2 +-
 .../apache/flink/table/plan/batch/sql/SubplanReuseTest.scala  |  2 +-
 .../flink/table/plan/batch/sql/agg/HashAggregateTest.scala    |  3 ++-
 .../flink/table/plan/batch/sql/agg/SortAggregateTest.scala    |  2 +-
 .../flink/table/plan/batch/sql/agg/WindowAggregateTest.scala  |  3 ++-
 .../table/plan/batch/sql/join/BroadcastHashJoinTest.scala     |  3 ++-
 .../plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala   |  3 ++-
 .../flink/table/plan/batch/sql/join/LookupJoinTest.scala      |  1 +
 .../flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala  |  2 +-
 .../plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala      |  2 +-
 .../table/plan/batch/sql/join/ShuffledHashJoinTest.scala      |  3 ++-
 .../plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala    |  3 ++-
 .../flink/table/plan/batch/sql/join/SortMergeJoinTest.scala   |  3 ++-
 .../table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala |  3 ++-
 .../apache/flink/table/plan/common/JoinReorderTestBase.scala  |  3 ++-
 .../table/plan/nodes/resource/ExecNodeResourceTest.scala      |  3 ++-
 .../table/plan/rules/logical/SplitAggregateRuleTest.scala     |  2 +-
 .../physical/batch/RemoveRedundantLocalHashAggRuleTest.scala  |  2 +-
 .../physical/batch/RemoveRedundantLocalSortAggRuleTest.scala  |  2 +-
 .../physical/stream/RetractionRulesWithTwoStageAggTest.scala  |  2 +-
 .../table/plan/stream/sql/MiniBatchIntervalInferTest.scala    |  3 ++-
 .../table/plan/stream/sql/ModifiedMonotonicityTest.scala      |  2 +-
 .../apache/flink/table/plan/stream/sql/SubplanReuseTest.scala |  2 +-
 .../flink/table/plan/stream/sql/agg/AggregateTest.scala       |  3 ++-
 .../table/plan/stream/sql/agg/DistinctAggregateTest.scala     |  2 +-
 .../table/plan/stream/sql/agg/TwoStageAggregateTest.scala     |  2 +-
 .../flink/table/plan/stream/table/TwoStageAggregateTest.scala |  3 ++-
 .../org/apache/flink/table/runtime/batch/sql/CalcITCase.scala |  3 ++-
 .../apache/flink/table/runtime/batch/sql/DecimalITCase.scala  |  3 ++-
 .../org/apache/flink/table/runtime/batch/sql/MiscITCase.scala |  2 +-
 .../flink/table/runtime/batch/sql/OverWindowITCase.scala      |  2 +-
 .../table/runtime/batch/sql/PartitionableSinkITCase.scala     |  4 +++-
 .../org/apache/flink/table/runtime/batch/sql/RankITCase.scala |  1 -
 .../flink/table/runtime/batch/sql/SortLimitITCase.scala       |  2 +-
 .../apache/flink/table/runtime/batch/sql/UnionITCase.scala    |  2 +-
 .../runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala  |  2 +-
 .../runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala |  5 +++--
 .../table/runtime/batch/sql/agg/GroupingSetsITCase.scala      |  2 +-
 .../flink/table/runtime/batch/sql/agg/HashAggITCase.scala     |  3 +--
 .../runtime/batch/sql/agg/HashDistinctAggregateITCase.scala   |  2 +-
 .../flink/table/runtime/batch/sql/agg/SortAggITCase.scala     |  2 +-
 .../runtime/batch/sql/agg/SortDistinctAggregateITCase.scala   |  2 +-
 .../flink/table/runtime/batch/sql/join/InnerJoinITCase.scala  |  2 +-
 .../flink/table/runtime/batch/sql/join/JoinITCase.scala       |  3 ++-
 .../flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala |  3 ++-
 .../flink/table/runtime/batch/table/DecimalITCase.scala       |  3 ++-
 .../flink/table/runtime/stream/sql/SplitAggregateITCase.scala |  2 +-
 .../org/apache/flink/table/runtime/utils/BatchTestBase.scala  |  3 ++-
 .../flink/table/runtime/utils/StreamingWithAggTestBase.scala  |  2 +-
 .../table/runtime/utils/StreamingWithMiniBatchTestBase.scala  |  2 +-
 .../scala/org/apache/flink/table/util/TableTestBase.scala     |  1 +
 flink-table/flink-table-runtime-blink/pom.xml                 |  6 ++++++
 .../flink/table/runtime/hashtable/BaseHybridHashTable.java    |  2 +-
 .../apache/flink/table/runtime/sort/BinaryExternalSorter.java |  2 +-
 .../flink/table/runtime/sort/BufferedKVExternalSorter.java    |  2 +-
 .../flink/table/runtime/hashtable/BinaryHashTableTest.java    |  2 +-
 .../flink/table/runtime/hashtable/LongHashTableTest.java      |  2 +-
 .../flink/table/runtime/sort/BinaryExternalSorterTest.java    |  2 +-
 .../table/runtime/sort/BufferedKVExternalSorterTest.java      |  2 +-
 104 files changed, 156 insertions(+), 108 deletions(-)

diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
similarity index 97%
rename from flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/ExecutionConfigOptions.java
rename to flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
index 283625c..7053f3f 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/api/ExecutionConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java
@@ -16,15 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.api;
+package org.apache.flink.table.api.config;
 
 import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.table.runtime.window.grouping.HeapWindowsGrouping;
 
 import static org.apache.flink.configuration.ConfigOptions.key;
 
 /**
  * This class holds configuration constants used by Flink's table module.
+ *
+ * <p>This is only used for the Blink planner.
  */
 public class ExecutionConfigOptions {
 
@@ -48,7 +49,7 @@ public class ExecutionConfigOptions {
 					.withDescription("Default limit when user don't set a limit after order by. ");
 
 	public static final ConfigOption<Integer> SQL_EXEC_SORT_FILE_HANDLES_MAX_NUM =
-			key("sql.exec.sort.file-handles.num.max")
+			key("sql.exec.sort.max-num-file-handles")
 					.defaultValue(128)
 					.withDescription("The maximal fan-in for external merge sort. It limits the number of file handles per operator. " +
 							"If it is too small, may cause intermediate merging. But if it is too large, " +
@@ -125,10 +126,10 @@ public class ExecutionConfigOptions {
 	// ------------------------------------------------------------------------
 
 	/**
-	 * See {@link HeapWindowsGrouping}.
+	 * See {@code org.apache.flink.table.runtime.window.grouping.HeapWindowsGrouping}.
 	 */
 	public static final ConfigOption<Integer> SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT =
-			key("sql.exec.window-agg.buffer-size.limit")
+			key("sql.exec.window-agg.buffer-size-limit")
 					.defaultValue(100 * 1000)
 					.withDescription("Sets the window elements buffer size limit used in group window agg operator.");
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/OptimizerConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
similarity index 98%
rename from flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/OptimizerConfigOptions.java
rename to flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
index a4674a7..8be671a 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/OptimizerConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.api;
+package org.apache.flink.table.api.config;
 
 import org.apache.flink.configuration.ConfigOption;
 
@@ -24,6 +24,8 @@ import static org.apache.flink.configuration.ConfigOptions.key;
 
 /**
  * This class holds configuration constants used by Flink's table planner module.
+ *
+ * <p>This is only used for the Blink planner.
  */
 public class OptimizerConfigOptions {
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
index 0cf5169..7d6e450 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
@@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.graph.StreamGraph;
 import org.apache.flink.streaming.api.transformations.ShuffleMode;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.delegation.Executor;
 import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil;
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResourceUtil.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResourceUtil.java
index 1f3fc35..acaf177 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResourceUtil.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/NodeResourceUtil.java
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.resource;
 
 import org.apache.flink.api.common.operators.ResourceSpec;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 
 /**
  * Deal with resource config for {@link org.apache.flink.table.plan.nodes.exec.ExecNode}.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/OperatorType.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/OperatorType.java
index f267981..5c972b8 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/OperatorType.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/util/OperatorType.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.util;
 
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 
 /**
  * Some dedicated operator type which is used in
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
index 315a7f3..b46ab56 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.util;
 
-import org.apache.flink.table.api.OptimizerConfigOptions;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
 
 /**
  * Aggregate phase strategy which could be specified in {@link OptimizerConfigOptions#SQL_OPTIMIZER_AGG_PHASE_STRATEGY}.
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
index 05abfe3..690eafb 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
@@ -29,8 +29,8 @@ import java.util.Set;
 
 import scala.concurrent.duration.Duration;
 
-import static org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS;
-import static org.apache.flink.table.api.OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY;
+import static org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS;
+import static org.apache.flink.table.api.config.OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY;
 
 /**
  * Utility class for {@link TableConfig} related helper functions.
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
index c2000f7..aa42bee 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.codegen
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.codegen.CodeGenUtils.{BASE_ROW, BINARY_ROW, DEFAULT_INPUT1_TERM, DEFAULT_INPUT2_TERM, className, newName}
 import org.apache.flink.table.codegen.OperatorCodeGenerator.{INPUT_SELECTION, generateCollect}
 import org.apache.flink.table.dataformat.{BaseRow, JoinedRow}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
index 45a6266..bde6b9c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
@@ -25,7 +25,8 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.streaming.api.operators.ProcessOperator
 import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig, TableException, TableSchema}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableException, TableSchema}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.LookupJoinCodeGenerator._
 import org.apache.flink.table.codegen.{CodeGeneratorContext, LookupJoinCodeGenerator}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
index 7befb58..fce4ee8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.nodes.logical
 
-import org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT
+import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
index f2170cb..75c471e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.transformations.{PartitionTransformation, ShuffleMode}
 import org.apache.flink.streaming.runtime.partitioner.{BroadcastPartitioner, GlobalPartitioner, RebalancePartitioner}
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.{CodeGeneratorContext, HashCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
index e395f63..0fa005e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
 import org.apache.flink.table.codegen.agg.batch.{AggWithoutKeysCodeGenerator, HashAggCodeGenerator}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
index c250c20..d6af5c6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.dag.Transformation
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.ProjectionCodeGenerator.generateProjection
 import org.apache.flink.table.codegen.{CodeGeneratorContext, LongHashJoinGenerator}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
index e9d49ad..b076e46 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
index 229bebc..62132d2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.{CodeGeneratorContext, NestedLoopJoinCodeGenerator}
 import org.apache.flink.table.dataformat.BaseRow
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
index 16c0c4d..0331186 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
@@ -22,7 +22,8 @@ import org.apache.flink.api.dag.Transformation
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
 import org.apache.flink.table.CalcitePair
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig}
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
index 7662afc..2babfb8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
@@ -21,7 +21,7 @@ import org.apache.flink.api.dag.Transformation
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.sort.SortCodeGenerator
 import org.apache.flink.table.dataformat.BaseRow
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
index bb04dd4..1808ba6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.runtime.operators.DamBehavior
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
 import org.apache.flink.table.codegen.ProjectionCodeGenerator.generateProjection
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
index ab8be01..c0157c9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.physical.batch
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.CodeGeneratorContext
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
index 581a9a4..01cacf9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
@@ -21,7 +21,8 @@ package org.apache.flink.table.plan.nodes.physical.stream
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.operators.KeyedProcessOperator
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.plan.rules.physical.stream.StreamExecRetractionRules
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
index a9ee7e8..1a9df7a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
@@ -19,7 +19,8 @@ package org.apache.flink.table.plan.nodes.physical.stream
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig, TableException}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.codegen.{CodeGeneratorContext, EqualiserCodeGenerator}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
index a9cd9f3..30afbe3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes.physical.stream
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.operators.KeyedProcessOperator
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator
 import org.apache.flink.table.codegen.{CodeGeneratorContext, EqualiserCodeGenerator}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
index 84359d1..dae7285 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
@@ -20,7 +20,8 @@ package org.apache.flink.table.plan.nodes.physical.stream
 
 import org.apache.flink.api.dag.Transformation
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.{FlinkContext, FlinkTypeFactory}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.`trait`.{MiniBatchIntervalTraitDef, MiniBatchMode}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
index c0196e9..20d4762 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.optimize
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig}
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.catalog.FunctionCatalog
 import org.apache.flink.table.plan.`trait`.{AccMode, AccModeTraitDef, MiniBatchInterval, MiniBatchIntervalTrait, MiniBatchIntervalTraitDef, MiniBatchMode, UpdateAsRetractionTraitDef}
 import org.apache.flink.table.plan.metadata.FlinkRelMetadataQuery
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkBatchProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkBatchProgram.scala
index edc15e3..d26cb40 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkBatchProgram.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkBatchProgram.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.optimize.program
 
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.rules.FlinkBatchRuleSets
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkStreamProgram.scala
index 7fac4cd..6c57bdb 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkStreamProgram.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/FlinkStreamProgram.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.optimize.program
 
 import org.apache.flink.configuration.Configuration
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.rules.{FlinkBatchRuleSets, FlinkStreamRuleSets}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
index 548efa0..bb0c726 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.reuse
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, TableConfig, TableException}
+import org.apache.flink.table.api.config.OptimizerConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.nodes.calcite.Sink
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableSourceScan
 import org.apache.flink.table.plan.nodes.physical.PhysicalTableSourceScan
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
index 14f173d..e73e8ec 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.expressions.{Expression, RexNodeConverter}
 import org.apache.flink.table.plan.schema.{FlinkRelOptTable, TableSourceTable}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
index 3e86e44..5a31709 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.TableException
 import org.apache.flink.table.calcite.{FlinkContext, FlinkLogicalRelFactories, FlinkRelBuilder}
 import org.apache.flink.table.functions.sql.{FlinkSqlOperatorTable, SqlFirstLastValueAggFunction}
 import org.apache.flink.table.plan.PartialFinalType
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
index dd2ddfd..9a24a16 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
index 82a280f..e1d2cea 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
@@ -19,7 +19,8 @@
 package org.apache.flink.table.plan.rules.physical.batch
 
 import org.apache.flink.table.JDouble
-import org.apache.flink.table.api.{OptimizerConfigOptions, TableConfig}
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
index 40abde4..6a177f2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
index 7d1f4e4..c41e775 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecWindowAggregateRule.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, TableConfig, TableException}
+import org.apache.flink.table.api.config.OptimizerConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.calcite.{FlinkContext, FlinkTypeFactory}
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
 import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
index 3b49fa1..07a7a0d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.rules.physical.stream
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.plan.`trait`.{MiniBatchInterval, MiniBatchIntervalTrait, MiniBatchIntervalTraitDef, MiniBatchMode}
 import org.apache.flink.table.plan.nodes.physical.stream.{StreamExecDataStreamScan, StreamExecGroupWindowAggregate, StreamExecTableSourceScan, StreamExecWatermarkAssigner, StreamPhysicalRel}
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
index e008744..5bc5114 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.rules.physical.stream
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.calcite.{FlinkContext, FlinkTypeFactory}
 import org.apache.flink.table.plan.`trait`.{AccMode, AccModeTrait, FlinkRelDistribution, FlinkRelDistributionTraitDef}
 import org.apache.flink.table.plan.metadata.FlinkRelMetadataQuery
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
index 933ec38..d73fe33 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
@@ -19,7 +19,8 @@ package org.apache.flink.table.plan.util
 
 import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.table.JLong
-import org.apache.flink.table.api.{DataTypes, ExecutionConfigOptions, TableConfig, TableException}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{DataTypes, TableConfig, TableException}
 import org.apache.flink.table.calcite.FlinkRelBuilder.PlannerNamedWindowProperty
 import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkTypeSystem}
 import org.apache.flink.table.dataformat.BaseRow
@@ -43,6 +44,7 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot
 import org.apache.flink.table.types.logical.{LogicalTypeRoot, _}
 import org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType
 import org.apache.flink.table.typeutils.BinaryStringTypeInfo
+
 import org.apache.calcite.rel.`type`._
 import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
 import org.apache.calcite.rex.RexInputRef
@@ -50,6 +52,7 @@ import org.apache.calcite.sql.fun._
 import org.apache.calcite.sql.validate.SqlMonotonicity
 import org.apache.calcite.sql.{SqlKind, SqlRankFunction}
 import org.apache.calcite.tools.RelBuilder
+
 import java.time.Duration
 import java.util
 
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
index 203920d..9e4f46e 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/plan/nodes/resource/parallelism/ShuffleStageParallelismCalculatorTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.nodes.resource.parallelism;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecCalc;
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecTableSourceScan;
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
index 12dd76a..f253a70 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.api.batch
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 import org.apache.flink.table.util.TableTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
index 83a51fb..e4a92d8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.api.stream
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 import org.apache.flink.table.util.TableTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala
index b33ebfb..7805b63 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/catalog/CatalogTableITCase.scala
@@ -18,8 +18,9 @@
 
 package org.apache.flink.table.catalog
 
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.internal.TableEnvironmentImpl
-import org.apache.flink.table.api.{EnvironmentSettings, ExecutionConfigOptions, TableEnvironment, TableException}
+import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment, TableException}
 import org.apache.flink.table.factories.utils.TestCollectionTableFactory
 import org.apache.flink.types.Row
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
index c815497..6f2daa5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.{Before, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
index 3969a6d..55823a4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
@@ -20,7 +20,8 @@ package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, Types}
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortMergeJoinRule
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
index ad38673..db8ab5c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
@@ -20,7 +20,8 @@ package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions, Types}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.Types
 import org.apache.flink.table.plan.rules.physical.batch.{BatchExecJoinRuleBase, BatchExecSortMergeJoinRule}
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 import org.apache.flink.table.util.{TableFunc1, TableTestBase}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
index 6998381..50ab944 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
@@ -22,7 +22,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.GenericTypeInfo
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, ValidationException}
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.plan.util.NonPojo
 import org.apache.flink.table.util.TableTestBase
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
index 72ec274..b9a119e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.util.TableTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
index 22f4908..546f965 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.util.TableTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
index 7288a5b..867ee8f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.IntFirstValueAggFunction
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction
 import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortMergeJoinRule
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
index 2f1d077..2b8095c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
@@ -17,7 +17,8 @@
  */
 package org.apache.flink.table.plan.batch.sql.agg
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions, TableException}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.TableException
 import org.apache.flink.table.plan.util.OperatorType
 import org.apache.flink.table.util.AggregatePhaseStrategy
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
index 0bbed95..dfdff36 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.plan.batch.sql.agg
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.plan.util.OperatorType
 import org.apache.flink.table.util.AggregatePhaseStrategy
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
index 52bfff5..43d5eab 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
@@ -18,8 +18,9 @@
 package org.apache.flink.table.plan.batch.sql.agg
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{OptimizerConfigOptions, TableException, ValidationException}
+import org.apache.flink.table.api.{TableException, ValidationException}
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMerge
 import org.apache.flink.table.util.{AggregatePhaseStrategy, CountAggFunction, TableTestBase}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
index 756a151..2c1d226 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.TableException
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
index 69b93d5..391bd64 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.TableException
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
index 10ea8a3..ca5fcaa 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
@@ -19,6 +19,7 @@ package org.apache.flink.table.plan.batch.sql.join
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api._
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
 import org.apache.flink.table.plan.stream.sql.join.TestTemporalTable
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
index 8df75db..6106320 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.Before
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
index d152961..7cf898e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.Before
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
index e4703f7..6dba826 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
index 0089486..6fc8716 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
index 5a97ee7..332e3ee 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
index f676f03..10e8416 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.plan.batch.sql.join
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 import org.junit.{Before, Test}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
index 32cba57..6465c2b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
@@ -19,7 +19,8 @@
 package org.apache.flink.table.plan.common
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.api.{OptimizerConfigOptions, Types}
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.plan.stats.{ColumnStats, FlinkStatistic, TableStats}
 import org.apache.flink.table.util.{TableTestBase, TableTestUtil}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
index 3d0d4d7..5b13415 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
@@ -24,7 +24,8 @@ import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink}
 import org.apache.flink.streaming.api.environment
 import org.apache.flink.streaming.api.operators.StreamOperatorFactory
 import org.apache.flink.streaming.api.transformations.{SinkTransformation, SourceTransformation}
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig, TableSchema, Types}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableSchema, Types}
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 import org.apache.flink.table.sinks.{AppendStreamTableSink, StreamTableSink, TableSink}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
index 75dc85e..869eb94 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.FlinkStreamProgram
 import org.apache.flink.table.util.TableTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
index 03564ba..b6dcd84 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.plan.rules.physical.batch
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.{Before, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
index 775de9d..511b8a1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.plan.rules.physical.batch
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.{Before, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
index d5c3408..284e277 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.plan.rules.physical.stream
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.util.{AggregatePhaseStrategy, TableTestBase}
 
 import org.junit.{Before, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
index 4d79cfb..0640dec 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
@@ -20,7 +20,8 @@ package org.apache.flink.table.plan.stream.sql
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig}
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.util.WindowEmitStrategy.{SQL_EXEC_EMIT_EARLY_FIRE_DELAY, SQL_EXEC_EMIT_EARLY_FIRE_ENABLED}
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
index dd11055..b9f4238 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.plan.stream.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.plan.`trait`.RelModifiedMonotonicity
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
index 53aa95c..c83b270 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.stream.sql
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.IntFirstValueAggFunction
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction
 import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
index 0cca30a..527e0e7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
@@ -20,8 +20,9 @@ package org.apache.flink.table.plan.stream.sql.agg
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException, Types, ValidationException}
+import org.apache.flink.table.api.{TableException, Types, ValidationException}
 import org.apache.flink.table.typeutils.DecimalTypeInfo
 import org.apache.flink.table.util.{StreamTableTestUtil, TableTestBase}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
index 8772500..5e97fa1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.stream.sql.agg
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.rules.physical.stream.IncrementalAggregateRule
 import org.apache.flink.table.util.{AggregatePhaseStrategy, StreamTableTestUtil, TableTestBase}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
index 5eca0cb..3446a42 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.stream.sql.agg
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.util.{AggregatePhaseStrategy, TableTestBase}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TwoStageAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TwoStageAggregateTest.scala
index 902387f..6bd2e96 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/table/TwoStageAggregateTest.scala
@@ -20,8 +20,9 @@ package org.apache.flink.table.plan.stream.table
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{DataTypes, ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.DataTypes
 import org.apache.flink.table.util.{AggregatePhaseStrategy, StreamTableTestUtil, TableTestBase}
 
 import org.junit.{Before, Test}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
index 1e4279c..d28be4c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
@@ -26,7 +26,8 @@ import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.common.typeinfo.Types.INSTANT
 import org.apache.flink.api.java.typeutils._
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, ValidationException}
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.dataformat.DataFormatConverters.{LocalDateConverter, LocalDateTimeConverter}
 import org.apache.flink.table.dataformat.Decimal
 import org.apache.flink.table.expressions.utils.{RichFunc1, RichFunc2, RichFunc3, SplitUDF}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
index 137d277..82b5ecb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
@@ -19,7 +19,8 @@
 package org.apache.flink.table.runtime.batch.sql
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{DataTypes, ExecutionConfigOptions}
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
index 7fd55e3..275bd26 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM
+import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM
 import org.apache.flink.table.runtime.batch.sql.join.JoinITCaseHelper
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.SortMergeJoin
 import org.apache.flink.table.runtime.utils.BatchTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
index 342f82b..d48c918 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/OverWindowITCase.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.tuple.{Tuple1 => JTuple1}
 import org.apache.flink.api.java.typeutils.{RowTypeInfo, TupleTypeInfo}
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, Types}
+import org.apache.flink.table.api.Types
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala
index fe3918e..2cf128d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/PartitionableSinkITCase.scala
@@ -26,7 +26,8 @@ import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl
 import org.apache.flink.sql.parser.validate.FlinkSqlConformance
 import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink}
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig, TableException, TableSchema, ValidationException}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{TableConfig, TableSchema, ValidationException}
 import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.runtime.batch.sql.PartitionableSinkITCase._
 import org.apache.flink.table.runtime.utils.BatchTestBase
@@ -35,6 +36,7 @@ import org.apache.flink.table.runtime.utils.TestData._
 import org.apache.flink.table.sinks.{PartitionableTableSink, StreamTableSink, TableSink}
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 import org.apache.flink.types.Row
+
 import org.apache.calcite.config.Lex
 import org.apache.calcite.sql.parser.SqlParser
 import org.junit.Assert._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
index 1e5975c..76f3d5e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/RankITCase.scala
@@ -18,7 +18,6 @@
 
 package org.apache.flink.table.runtime.batch.sql
 
-import org.apache.flink.table.api.ExecutionConfigOptions
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.TestData._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
index d9faca9..5e25f17 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.runtime.batch.sql
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.TestData._
 import org.apache.flink.types.Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
index e8d9082..43b2108 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.runtime.batch.sql
 
-import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.dataformat.BinaryString.fromString
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.{binaryRow, row}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
index f6aa573..f523d0e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS
+import org.apache.flink.table.api.config.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS
 import org.apache.flink.table.api.{TableException, Types}
 import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
index 7a4b648..fcbc52e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
@@ -18,8 +18,9 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{OptimizerConfigOptions, Types}
-import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM, SQL_RESOURCE_HASH_AGG_TABLE_MEM}
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM, SQL_RESOURCE_HASH_AGG_TABLE_MEM}
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime
 import org.apache.flink.table.runtime.utils.BatchTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/GroupingSetsITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/GroupingSetsITCase.scala
index e7b8913c..eaa615c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/GroupingSetsITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/GroupingSetsITCase.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException, Types}
+import org.apache.flink.table.api.{TableException, Types}
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.TestData._
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
index 296e50d..416b7c5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
@@ -18,8 +18,7 @@
 
 package org.apache.flink.table.runtime.batch.sql.agg
 
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 
 /**
   * AggregateITCase using HashAgg Operator.
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
index 9218e55..3d199ab 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.runtime.batch.sql.agg
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.plan.util.OperatorType
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
index f40d0af..8d3faad 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.batch.sql.agg
 import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, PrimitiveArrayTypeInfo, TypeInformation}
 import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM}
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM}
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
index 78ea69c..72f2fd0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset
 import org.apache.flink.table.plan.util.OperatorType
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
index c8c35aa..b544835 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.runtime.batch.sql.join
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.runtime.batch.sql.join.JoinITCaseHelper.disableOtherJoinOpForJoin
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
 import org.apache.flink.table.runtime.utils.BatchTestBase
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
index f4544f1..216dca8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
@@ -22,7 +22,8 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO}
 import org.apache.flink.api.common.typeutils.TypeComparator
 import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo}
-import org.apache.flink.table.api.{ExecutionConfigOptions, Types}
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.expressions.utils.FuncWithOpen
 import org.apache.flink.table.runtime.CodeGenOperatorFactory
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
index de9cb7c..998bb28 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.runtime.batch.sql.join
 
-import org.apache.flink.table.api.{OptimizerConfigOptions, ExecutionConfigOptions, TableEnvironment}
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
 
 /**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
index 8bb6054..aa345d1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/table/DecimalITCase.scala
@@ -19,7 +19,8 @@
 package org.apache.flink.table.runtime.batch.table
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{DataTypes, ExecutionConfigOptions, Table}
+import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.{DataTypes, Table}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.{BatchTableEnvUtil, BatchTestBase}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
index 750086e..ee7580a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.runtime.stream.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.stream.sql.SplitAggregateITCase.PartialAggMode
 import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase.{AggMode, LocalGlobalOff, LocalGlobalOn}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
index 670bc94..e31971a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
@@ -23,8 +23,9 @@ import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.streaming.api.transformations.ShuffleMode
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.internal.{TableEnvironmentImpl, TableImpl}
-import org.apache.flink.table.api.{EnvironmentSettings, ExecutionConfigOptions, SqlParserException, Table, TableConfig, TableEnvironment}
+import org.apache.flink.table.api.{EnvironmentSettings, SqlParserException, Table, TableConfig, TableEnvironment}
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, BinaryRowWriter}
 import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
 import org.apache.flink.table.plan.stats.FlinkStatistic
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
index 0c49792..751688a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.runtime.utils
 
 import java.util
 import org.apache.flink.api.common.time.Time
-import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase._
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.{MiniBatchMode, MiniBatchOff, MiniBatchOn}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
index 0afccfe..4c90eb2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.runtime.utils
 
-import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_MINIBATCH_ALLOW_LATENCY, SQL_EXEC_MINIBATCH_SIZE, SQL_EXEC_MINIBATCH_ENABLED}
+import org.apache.flink.table.api.config.ExecutionConfigOptions.{SQL_EXEC_MINIBATCH_ALLOW_LATENCY, SQL_EXEC_MINIBATCH_SIZE, SQL_EXEC_MINIBATCH_ENABLED}
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.{MiniBatchMode, MiniBatchOff, MiniBatchOn}
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
index 1e88629..08b5506 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
@@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => Scala
 import org.apache.flink.streaming.api.transformations.ShuffleMode
 import org.apache.flink.streaming.api.{TimeCharacteristic, environment}
 import org.apache.flink.table.api._
+import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.internal.{TableEnvironmentImpl, TableImpl}
 import org.apache.flink.table.api.java.internal.{StreamTableEnvironmentImpl => JavaStreamTableEnvImpl}
 import org.apache.flink.table.api.java.{StreamTableEnvironment => JavaStreamTableEnv}
diff --git a/flink-table/flink-table-runtime-blink/pom.xml b/flink-table/flink-table-runtime-blink/pom.xml
index 5a86c18..f16d5a2 100644
--- a/flink-table/flink-table-runtime-blink/pom.xml
+++ b/flink-table/flink-table-runtime-blink/pom.xml
@@ -50,6 +50,12 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-api-java</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
index 1ac0c0d..2e71061 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/hashtable/BaseHybridHashTable.java
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.disk.iomanager.HeaderlessChannelReaderInputVi
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.runtime.compression.BlockCompressionFactory;
 import org.apache.flink.table.runtime.util.FileChannelUtil;
 import org.apache.flink.table.runtime.util.MemorySegmentPool;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
index 817a3f8..d752fdb 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BinaryExternalSorter.java
@@ -32,7 +32,7 @@ import org.apache.flink.runtime.operators.sort.IndexedSorter;
 import org.apache.flink.runtime.operators.sort.QuickSort;
 import org.apache.flink.runtime.operators.sort.Sorter;
 import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.generated.NormalizedKeyComputer;
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
index 326daf0..b8e7a69 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorter.java
@@ -25,7 +25,7 @@ import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.operators.sort.IndexedSorter;
 import org.apache.flink.runtime.operators.sort.QuickSort;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.generated.NormalizedKeyComputer;
 import org.apache.flink.table.generated.RecordComparator;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
index c760f02..1e4ebb0 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/BinaryHashTableTest.java
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.UnionIterator;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
index 6dcad8c..1933493 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/hashtable/LongHashTableTest.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.UnionIterator;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.runtime.util.RowIterator;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryExternalSorterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryExternalSorterTest.java
index 70cb5fb..44d663f 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryExternalSorterTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BinaryExternalSorterTest.java
@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BaseRow;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorterTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorterTest.java
index 7eedc34..15a4154b 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorterTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/sort/BufferedKVExternalSorterTest.java
@@ -25,7 +25,7 @@ import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.table.api.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.dataformat.BinaryRow;
 import org.apache.flink.table.dataformat.BinaryRowWriter;
 import org.apache.flink.table.dataformat.BinaryString;


[flink] 03/03: [FLINK-13266][table] Relocate blink planner classes to avoid class clashes

Posted by ja...@apache.org.
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 c601cfd662c2839f8ebc81b80879ecce55a8cbaf
Author: godfreyhe <go...@163.com>
AuthorDate: Mon Jul 22 13:42:40 2019 +0800

    [FLINK-13266][table] Relocate blink planner classes to avoid class clashes
---
 .../flink/table/api/EnvironmentSettings.java       |    4 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java    |    3 +-
 .../table/calcite/FlinkCalciteCatalogReader.java   |   78 -
 .../flink/table/calcite/type/FlinkReturnTypes.java |  154 -
 .../type/NumericExceptFirstOperandChecker.java     |   92 -
 .../type/NumericOrDefaultReturnTypeInference.java  |   68 -
 .../type/RepeatFamilyOperandTypeChecker.java       |  125 -
 .../flink/table/catalog/CatalogCalciteSchema.java  |   91 -
 .../table/catalog/CatalogManagerCalciteSchema.java |   91 -
 .../flink/table/catalog/DatabaseCalciteSchema.java |  199 --
 .../apache/flink/table/catalog/FlinkSchema.java    |   59 -
 .../catalog/FunctionCatalogOperatorTable.java      |  142 -
 .../catalog/QueryOperationCatalogViewTable.java    |  106 -
 .../apache/flink/table/executor/BatchExecutor.java |  110 -
 .../flink/table/executor/BlinkExecutorFactory.java |   81 -
 .../apache/flink/table/executor/ExecutorBase.java  |   72 -
 .../flink/table/executor/StreamExecutor.java       |   56 -
 .../flink/table/expressions/ExpressionBuilder.java |  143 -
 .../expressions/PlannerTypeInferenceUtilImpl.java  |  144 -
 .../expressions/ResolvedAggInputReference.java     |   96 -
 .../expressions/ResolvedAggLocalReference.java     |   98 -
 .../expressions/ResolvedDistinctKeyReference.java  |   88 -
 .../flink/table/expressions/RexNodeConverter.java  | 1015 -------
 .../flink/table/expressions/RexNodeExpression.java |   70 -
 .../table/expressions/SqlAggFunctionVisitor.java   |  100 -
 .../functions/InternalFunctionDefinitions.java     |   37 -
 .../functions/aggfunctions/AvgAggFunction.java     |  181 --
 .../functions/aggfunctions/CollectAggFunction.java |  131 -
 .../functions/aggfunctions/ConcatAggFunction.java  |  115 -
 .../aggfunctions/ConcatWithRetractAggFunction.java |  138 -
 .../ConcatWsWithRetractAggFunction.java            |  149 -
 .../functions/aggfunctions/Count1AggFunction.java  |   91 -
 .../functions/aggfunctions/CountAggFunction.java   |   92 -
 .../aggfunctions/DeclarativeAggregateFunction.java |  172 --
 .../aggfunctions/DenseRankAggFunction.java         |   83 -
 .../aggfunctions/FirstValueAggFunction.java        |  221 --
 .../FirstValueWithRetractAggFunction.java          |  408 ---
 .../functions/aggfunctions/IncrSumAggFunction.java |  172 --
 .../IncrSumWithRetractAggFunction.java             |  225 --
 .../aggfunctions/LastValueAggFunction.java         |  220 --
 .../LastValueWithRetractAggFunction.java           |  407 ---
 .../functions/aggfunctions/LeadLagAggFunction.java |  291 --
 .../functions/aggfunctions/MaxAggFunction.java     |  220 --
 .../aggfunctions/MaxWithRetractAggFunction.java    |  405 ---
 .../functions/aggfunctions/MinAggFunction.java     |  220 --
 .../aggfunctions/MinWithRetractAggFunction.java    |  405 ---
 .../functions/aggfunctions/RankAggFunction.java    |   96 -
 .../aggfunctions/RankLikeAggFunctionBase.java      |  131 -
 .../aggfunctions/RowNumberAggFunction.java         |   85 -
 .../aggfunctions/SingleValueAggFunction.java       |  282 --
 .../functions/aggfunctions/Sum0AggFunction.java    |  210 --
 .../functions/aggfunctions/SumAggFunction.java     |  168 --
 .../aggfunctions/SumWithRetractAggFunction.java    |  222 --
 .../table/functions/sql/CalciteSqlFunction.java    |   69 -
 .../table/functions/sql/FlinkSqlOperatorTable.java | 1196 --------
 .../sql/ProctimeMaterializeSqlFunction.java        |   66 -
 .../table/functions/sql/SqlConcatAggFunction.java  |   66 -
 .../sql/SqlFirstLastValueAggFunction.java          |   65 -
 .../table/functions/sql/SqlIncrSumAggFunction.java |   75 -
 .../functions/sql/SqlThrowExceptionFunction.java   |   41 -
 .../sql/StreamRecordTimestampSqlFunction.java      |   55 -
 .../sql/internal/SqlAuxiliaryGroupAggFunction.java |   47 -
 .../functions/tablefunctions/ReplicateRows.java    |   71 -
 .../table/operations/DataStreamQueryOperation.java |  140 -
 .../table/operations/PlannerQueryOperation.java    |   84 -
 .../operations/RichTableSourceQueryOperation.java  |   74 -
 .../apache/flink/table/plan/PartialFinalType.java  |   41 -
 .../flink/table/plan/QueryOperationConverter.java  |  552 ----
 .../flink/table/plan/metadata/FlinkMetadata.java   |  243 --
 .../table/plan/metadata/FlinkRelMetadataQuery.java |  250 --
 .../table/plan/nodes/exec/ExecNodeVisitor.java     |   33 -
 .../table/plan/nodes/exec/ExecNodeVisitorImpl.java |   33 -
 .../plan/nodes/process/DAGProcessContext.java      |   42 -
 .../table/plan/nodes/process/DAGProcessor.java     |   34 -
 .../table/plan/nodes/resource/NodeResource.java    |   57 -
 .../plan/nodes/resource/NodeResourceUtil.java      |   77 -
 .../parallelism/FinalParallelismSetter.java        |  135 -
 .../resource/parallelism/ParallelismProcessor.java |   73 -
 .../nodes/resource/parallelism/ShuffleStage.java   |   88 -
 .../parallelism/ShuffleStageGenerator.java         |  157 -
 .../ShuffleStageParallelismCalculator.java         |   92 -
 ...FlinkAggregateExpandDistinctAggregatesRule.java |  925 ------
 .../logical/FlinkAggregateJoinTransposeRule.java   |  554 ----
 .../rules/logical/FlinkAggregateRemoveRule.java    |  131 -
 .../plan/rules/logical/FlinkFilterJoinRule.java    |  363 ---
 .../logical/FlinkJoinPushExpressionsRule.java      |   83 -
 .../logical/FlinkProjectJoinTransposeRule.java     |  151 -
 .../FlinkSemiAntiJoinFilterTransposeRule.java      |   91 -
 .../FlinkSemiAntiJoinJoinTransposeRule.java        |  281 --
 .../FlinkSemiAntiJoinProjectTransposeRule.java     |  202 --
 .../plan/rules/logical/SubQueryDecorrelator.java   | 1440 ---------
 .../apache/flink/table/plan/util/JoinTypeUtil.java |   52 -
 .../flink/table/plan/util/KeySelectorUtil.java     |   68 -
 .../apache/flink/table/plan/util/OperatorType.java |   40 -
 .../flink/table/planner/BlinkPlannerFactory.java   |   72 -
 .../apache/flink/table/planner/PlannerContext.java |  236 --
 .../planner/catalog/CatalogCalciteSchema.java      |   92 +
 .../catalog/CatalogManagerCalciteSchema.java       |   94 +
 .../planner/catalog/DatabaseCalciteSchema.java     |  205 ++
 .../flink/table/planner/catalog/FlinkSchema.java   |   59 +
 .../catalog/FunctionCatalogOperatorTable.java      |  144 +
 .../catalog/QueryOperationCatalogViewTable.java    |  107 +
 .../table/planner/delegation/BatchExecutor.java    |  110 +
 .../planner/delegation/BlinkExecutorFactory.java   |   81 +
 .../planner/delegation/BlinkPlannerFactory.java    |   72 +
 .../table/planner/delegation/ExecutorBase.java     |   72 +
 .../table/planner/delegation/PlannerContext.java   |  236 ++
 .../table/planner/delegation/StreamExecutor.java   |   56 +
 .../planner/expressions/ExpressionBuilder.java     |  145 +
 .../expressions/PlannerTypeInferenceUtilImpl.java  |  147 +
 .../expressions/ResolvedAggInputReference.java     |   99 +
 .../expressions/ResolvedAggLocalReference.java     |  100 +
 .../expressions/ResolvedDistinctKeyReference.java  |   90 +
 .../planner/expressions/RexNodeConverter.java      | 1029 +++++++
 .../planner/expressions/RexNodeExpression.java     |   73 +
 .../planner/expressions/SqlAggFunctionVisitor.java |  103 +
 .../functions/InternalFunctionDefinitions.java     |   38 +
 .../functions/aggfunctions/AvgAggFunction.java     |  181 ++
 .../functions/aggfunctions/CollectAggFunction.java |  131 +
 .../functions/aggfunctions/ConcatAggFunction.java  |  115 +
 .../aggfunctions/ConcatWithRetractAggFunction.java |  138 +
 .../ConcatWsWithRetractAggFunction.java            |  149 +
 .../functions/aggfunctions/Count1AggFunction.java  |   91 +
 .../functions/aggfunctions/CountAggFunction.java   |   92 +
 .../aggfunctions/DeclarativeAggregateFunction.java |  172 ++
 .../aggfunctions/DenseRankAggFunction.java         |   83 +
 .../aggfunctions/FirstValueAggFunction.java        |  221 ++
 .../FirstValueWithRetractAggFunction.java          |  408 +++
 .../functions/aggfunctions/IncrSumAggFunction.java |  172 ++
 .../IncrSumWithRetractAggFunction.java             |  225 ++
 .../aggfunctions/LastValueAggFunction.java         |  220 ++
 .../LastValueWithRetractAggFunction.java           |  407 +++
 .../functions/aggfunctions/LeadLagAggFunction.java |  291 ++
 .../functions/aggfunctions/MaxAggFunction.java     |  220 ++
 .../aggfunctions/MaxWithRetractAggFunction.java    |  405 +++
 .../functions/aggfunctions/MinAggFunction.java     |  220 ++
 .../aggfunctions/MinWithRetractAggFunction.java    |  405 +++
 .../functions/aggfunctions/RankAggFunction.java    |   96 +
 .../aggfunctions/RankLikeAggFunctionBase.java      |  131 +
 .../aggfunctions/RowNumberAggFunction.java         |   85 +
 .../aggfunctions/SingleValueAggFunction.java       |  282 ++
 .../functions/aggfunctions/Sum0AggFunction.java    |  210 ++
 .../functions/aggfunctions/SumAggFunction.java     |  168 ++
 .../aggfunctions/SumWithRetractAggFunction.java    |  222 ++
 .../planner/functions/sql/CalciteSqlFunction.java  |   69 +
 .../functions/sql/FlinkSqlOperatorTable.java       | 1196 ++++++++
 .../sql/ProctimeMaterializeSqlFunction.java        |   66 +
 .../functions/sql/SqlConcatAggFunction.java        |   66 +
 .../sql/SqlFirstLastValueAggFunction.java          |   65 +
 .../functions/sql/SqlIncrSumAggFunction.java       |   75 +
 .../functions/sql/SqlThrowExceptionFunction.java   |   41 +
 .../sql/StreamRecordTimestampSqlFunction.java      |   55 +
 .../sql/internal/SqlAuxiliaryGroupAggFunction.java |   47 +
 .../functions/tablefunctions/ReplicateRows.java    |   71 +
 .../operations/DataStreamQueryOperation.java       |  144 +
 .../planner/operations/PlannerQueryOperation.java  |   88 +
 .../operations/RichTableSourceQueryOperation.java  |   77 +
 .../planner/operations/SqlConversionException.java |   35 +
 .../operations/SqlToOperationConverter.java        |  215 ++
 .../planner/plan/FlinkCalciteCatalogReader.java    |   78 +
 .../flink/table/planner/plan/PartialFinalType.java |   41 +
 .../planner/plan/QueryOperationConverter.java      |  552 ++++
 .../table/planner/plan/metadata/FlinkMetadata.java |  243 ++
 .../plan/metadata/FlinkRelMetadataQuery.java       |  250 ++
 .../planner/plan/nodes/exec/ExecNodeVisitor.java   |   33 +
 .../plan/nodes/exec/ExecNodeVisitorImpl.java       |   33 +
 .../plan/nodes/process/DAGProcessContext.java      |   42 +
 .../planner/plan/nodes/process/DAGProcessor.java   |   34 +
 .../planner/plan/nodes/resource/NodeResource.java  |   57 +
 .../plan/nodes/resource/NodeResourceUtil.java      |   77 +
 .../parallelism/FinalParallelismSetter.java        |  135 +
 .../resource/parallelism/ParallelismProcessor.java |   73 +
 .../nodes/resource/parallelism/ShuffleStage.java   |   88 +
 .../parallelism/ShuffleStageGenerator.java         |  157 +
 .../ShuffleStageParallelismCalculator.java         |   92 +
 ...FlinkAggregateExpandDistinctAggregatesRule.java |  925 ++++++
 .../logical/FlinkAggregateJoinTransposeRule.java   |  554 ++++
 .../rules/logical/FlinkAggregateRemoveRule.java    |  131 +
 .../plan/rules/logical/FlinkFilterJoinRule.java    |  363 +++
 .../logical/FlinkJoinPushExpressionsRule.java      |   83 +
 .../logical/FlinkProjectJoinTransposeRule.java     |  151 +
 .../FlinkSemiAntiJoinFilterTransposeRule.java      |   91 +
 .../FlinkSemiAntiJoinJoinTransposeRule.java        |  281 ++
 .../FlinkSemiAntiJoinProjectTransposeRule.java     |  202 ++
 .../plan/rules/logical/SubQueryDecorrelator.java   | 1440 +++++++++
 .../table/planner/plan/type/FlinkReturnTypes.java  |  154 +
 .../type/NumericExceptFirstOperandChecker.java     |   92 +
 .../type/NumericOrDefaultReturnTypeInference.java  |   68 +
 .../plan/type/RepeatFamilyOperandTypeChecker.java  |  125 +
 .../table/planner/plan/utils/JoinTypeUtil.java     |   52 +
 .../table/planner/plan/utils/KeySelectorUtil.java  |   68 +
 .../table/planner/plan/utils/OperatorType.java     |   40 +
 .../planner/utils/AggregatePhaseStrategy.java      |   43 +
 .../table/planner/utils/SingleElementIterator.java |   69 +
 .../table/planner/utils/TableConfigUtils.java      |  118 +
 .../table/sqlexec/SqlConversionException.java      |   35 -
 .../table/sqlexec/SqlToOperationConverter.java     |  216 --
 .../flink/table/util/AggregatePhaseStrategy.java   |   43 -
 .../flink/table/util/SingleElementIterator.java    |   69 -
 .../apache/flink/table/util/TableConfigUtils.java  |  118 -
 .../org.apache.flink.table.factories.TableFactory  |    4 +-
 .../apache/flink/table/calcite/CalciteConfig.scala |  228 --
 .../table/calcite/FlinkCalciteSqlValidator.scala   |   95 -
 .../apache/flink/table/calcite/FlinkContext.scala  |   45 -
 .../flink/table/calcite/FlinkContextImpl.scala     |   32 -
 .../apache/flink/table/calcite/FlinkLocalRef.scala |   48 -
 .../table/calcite/FlinkLogicalRelFactories.scala   |  262 --
 .../flink/table/calcite/FlinkPlannerImpl.scala     |  210 --
 .../flink/table/calcite/FlinkRelBuilder.scala      |  169 --
 .../flink/table/calcite/FlinkRelFactories.scala    |  124 -
 .../flink/table/calcite/FlinkRelOptCluster.scala   |   77 -
 .../table/calcite/FlinkRelOptClusterFactory.scala  |   43 -
 .../flink/table/calcite/FlinkTypeFactory.scala     |  518 ----
 .../flink/table/calcite/FlinkTypeSystem.scala      |  148 -
 .../flink/table/calcite/PreValidateReWriter.scala  |  201 --
 .../table/calcite/RelTimeIndicatorConverter.scala  |  700 -----
 .../flink/table/codegen/CalcCodeGenerator.scala    |  215 --
 .../flink/table/codegen/CodeGenException.scala     |   24 -
 .../apache/flink/table/codegen/CodeGenUtils.scala  |  721 -----
 .../flink/table/codegen/CodeGeneratorContext.scala |  830 -----
 .../table/codegen/CollectorCodeGenerator.scala     |   98 -
 .../table/codegen/CorrelateCodeGenerator.scala     |  427 ---
 .../table/codegen/EqualiserCodeGenerator.scala     |  147 -
 .../flink/table/codegen/ExpandCodeGenerator.scala  |   77 -
 .../flink/table/codegen/ExprCodeGenerator.scala    |  756 -----
 .../flink/table/codegen/ExpressionReducer.scala    |  255 --
 .../table/codegen/FunctionCodeGenerator.scala      |  218 --
 .../apache/flink/table/codegen/GenerateUtils.scala |  827 -----
 .../flink/table/codegen/GeneratedExpression.scala  |  103 -
 .../flink/table/codegen/HashCodeGenerator.scala    |  106 -
 .../org/apache/flink/table/codegen/Indenter.scala  |   58 -
 .../table/codegen/InputFormatCodeGenerator.scala   |   93 -
 .../table/codegen/LongHashJoinGenerator.scala      |  359 ---
 .../table/codegen/LookupJoinCodeGenerator.scala    |  456 ---
 .../flink/table/codegen/MatchCodeGenerator.scala   |  814 -----
 .../codegen/NestedLoopJoinCodeGenerator.scala      |  370 ---
 .../table/codegen/OperatorCodeGenerator.scala      |  232 --
 .../table/codegen/ProjectionCodeGenerator.scala    |  214 --
 .../flink/table/codegen/SinkCodeGenerator.scala    |  275 --
 .../flink/table/codegen/ValuesCodeGenerator.scala  |   60 -
 .../flink/table/codegen/agg/AggCodeGen.scala       |   49 -
 .../codegen/agg/AggsHandlerCodeGenerator.scala     |  824 -----
 .../table/codegen/agg/DeclarativeAggCodeGen.scala  |  308 --
 .../table/codegen/agg/DistinctAggCodeGen.scala     |  934 ------
 .../table/codegen/agg/ImperativeAggCodeGen.scala   |  506 ----
 .../table/codegen/agg/batch/AggCodeGenHelper.scala |  716 -----
 .../agg/batch/AggWithoutKeysCodeGenerator.scala    |  118 -
 .../codegen/agg/batch/HashAggCodeGenHelper.scala   |  845 ------
 .../codegen/agg/batch/HashAggCodeGenerator.scala   |  230 --
 .../agg/batch/HashWindowCodeGenerator.scala        |  815 -----
 .../codegen/agg/batch/SortAggCodeGenerator.scala   |  149 -
 .../agg/batch/SortWindowCodeGenerator.scala        |  269 --
 .../codegen/agg/batch/WindowCodeGenerator.scala    |  764 -----
 .../flink/table/codegen/calls/BuiltInMethods.scala |  493 ---
 .../flink/table/codegen/calls/CallGenerator.scala  |   35 -
 .../table/codegen/calls/ConstantCallGen.scala      |   37 -
 .../codegen/calls/CurrentTimePointCallGen.scala    |   58 -
 .../flink/table/codegen/calls/DivCallGen.scala     |   67 -
 .../flink/table/codegen/calls/ExtractCallGen.scala |  115 -
 .../table/codegen/calls/FloorCeilCallGen.scala     |   93 -
 .../table/codegen/calls/FunctionGenerator.scala    |  923 ------
 .../table/codegen/calls/HashCodeCallGen.scala      |   36 -
 .../flink/table/codegen/calls/IfCallGen.scala      |   67 -
 .../flink/table/codegen/calls/LikeCallGen.scala    |  171 --
 .../flink/table/codegen/calls/MethodCallGen.scala  |   70 -
 .../flink/table/codegen/calls/PrintCallGen.scala   |   67 -
 .../flink/table/codegen/calls/RandCallGen.scala    |   56 -
 .../codegen/calls/ScalarFunctionCallGen.scala      |  138 -
 .../table/codegen/calls/ScalarOperatorGens.scala   | 2177 --------------
 .../flink/table/codegen/calls/StringCallGen.scala  |  814 -----
 .../table/codegen/calls/TableFunctionCallGen.scala |   93 -
 .../table/codegen/calls/TimestampDiffCallGen.scala |  120 -
 ...ltiFieldRangeBoundComparatorCodeGenerator.scala |   84 -
 .../over/RangeBoundComparatorCodeGenerator.scala   |  177 --
 .../codegen/sort/ComparatorCodeGenerator.scala     |   84 -
 .../table/codegen/sort/SortCodeGenerator.scala     |  464 ---
 .../apache/flink/table/dataview/DataViewSpec.scala |   47 -
 .../flink/table/dataview/DataViewUtils.scala       |  215 --
 .../flink/table/expressions/ExpressionBridge.scala |   40 -
 .../flink/table/expressions/InputTypeSpec.scala    |   69 -
 .../table/expressions/PlannerExpression.scala      |   87 -
 .../expressions/PlannerExpressionConverter.scala   |  848 ------
 .../expressions/PlannerExpressionParserImpl.scala  |    8 +-
 .../table/expressions/PlannerExpressionUtils.scala |   68 -
 .../flink/table/expressions/Reinterpret.scala      |   45 -
 .../table/expressions/ReturnTypeInference.scala    |  218 --
 .../flink/table/expressions/aggregations.scala     |  230 --
 .../flink/table/expressions/arithmetic.scala       |  149 -
 .../org/apache/flink/table/expressions/call.scala  |  223 --
 .../org/apache/flink/table/expressions/cast.scala  |   44 -
 .../flink/table/expressions/collection.scala       |  174 --
 .../flink/table/expressions/comparison.scala       |  181 --
 .../apache/flink/table/expressions/composite.scala |  100 -
 .../flink/table/expressions/fieldExpression.scala  |  230 --
 .../flink/table/expressions/hashExpressions.scala  |   93 -
 .../apache/flink/table/expressions/literals.scala  |   82 -
 .../org/apache/flink/table/expressions/logic.scala |   87 -
 .../flink/table/expressions/mathExpressions.scala  |  401 ---
 .../apache/flink/table/expressions/ordering.scala  |   43 -
 .../flink/table/expressions/overOffsets.scala      |   54 -
 .../expressions/plannerWindowProperties.scala      |   86 -
 .../table/expressions/stringExpressions.scala      |  490 ---
 .../apache/flink/table/expressions/subquery.scala  |   76 -
 .../apache/flink/table/expressions/symbols.scala   |  127 -
 .../org/apache/flink/table/expressions/time.scala  |  322 --
 .../flink/table/expressions/windowProperties.scala |   61 -
 .../table/functions/utils/AggSqlFunction.scala     |  216 --
 .../table/functions/utils/ScalarSqlFunction.scala  |  199 --
 .../table/functions/utils/TableSqlFunction.scala   |  191 --
 .../functions/utils/UserDefinedFunctionUtils.scala |  858 ------
 .../scala/org/apache/flink/table/package.scala     |   43 -
 .../org/apache/flink/table/plan/TreeNode.scala     |  115 -
 .../apache/flink/table/plan/cost/FlinkCost.scala   |  354 ---
 .../flink/table/plan/cost/FlinkCostBase.scala      |   37 -
 .../flink/table/plan/cost/FlinkCostFactory.scala   |   49 -
 .../table/plan/cost/FlinkCostFactoryBase.scala     |   35 -
 .../flink/table/plan/logical/MatchRecognize.scala  |   42 -
 .../flink/table/plan/logical/groupWindows.scala    |   78 -
 .../metadata/AggCallSelectivityEstimator.scala     |  455 ---
 .../metadata/FlinkDefaultRelMetadataProvider.scala |   53 -
 .../plan/metadata/FlinkRelMdColumnInterval.scala   |  746 -----
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |  295 --
 .../metadata/FlinkRelMdColumnOriginNullCount.scala |  134 -
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |  739 -----
 .../plan/metadata/FlinkRelMdCumulativeCost.scala   |   59 -
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |  626 ----
 .../plan/metadata/FlinkRelMdDistribution.scala     |  103 -
 .../FlinkRelMdFilteredColumnInterval.scala         |  264 --
 .../metadata/FlinkRelMdModifiedMonotonicity.scala  |  552 ----
 .../metadata/FlinkRelMdNonCumulativeCost.scala     |   55 -
 .../FlinkRelMdPercentageOriginalRows.scala         |  146 -
 .../plan/metadata/FlinkRelMdPopulationSize.scala   |  412 ---
 .../table/plan/metadata/FlinkRelMdRowCount.scala   |  457 ---
 .../plan/metadata/FlinkRelMdSelectivity.scala      |  301 --
 .../flink/table/plan/metadata/FlinkRelMdSize.scala |  436 ---
 .../plan/metadata/FlinkRelMdUniqueGroups.scala     |  432 ---
 .../table/plan/metadata/FlinkRelMdUniqueKeys.scala |  566 ----
 .../table/plan/metadata/SelectivityEstimator.scala | 1146 -------
 .../flink/table/plan/nodes/FlinkConventions.scala  |   55 -
 .../flink/table/plan/nodes/FlinkRelNode.scala      |  118 -
 .../flink/table/plan/nodes/calcite/Expand.scala    |  109 -
 .../table/plan/nodes/calcite/LogicalExpand.scala   |   58 -
 .../table/plan/nodes/calcite/LogicalRank.scala     |   96 -
 .../table/plan/nodes/calcite/LogicalSink.scala     |   57 -
 .../nodes/calcite/LogicalWatermarkAssigner.scala   |   43 -
 .../nodes/calcite/LogicalWindowAggregate.scala     |   89 -
 .../flink/table/plan/nodes/calcite/Rank.scala      |  137 -
 .../flink/table/plan/nodes/calcite/Sink.scala      |   58 -
 .../plan/nodes/calcite/WatermarkAssigner.scala     |   65 -
 .../table/plan/nodes/calcite/WindowAggregate.scala |   87 -
 .../flink/table/plan/nodes/common/CommonCalc.scala |   86 -
 .../nodes/common/CommonIntermediateTableScan.scala |   54 -
 .../table/plan/nodes/common/CommonLookupJoin.scala |  678 -----
 .../plan/nodes/common/CommonPhysicalExchange.scala |  109 -
 .../plan/nodes/common/CommonPhysicalJoin.scala     |   86 -
 .../table/plan/nodes/exec/BatchExecNode.scala      |   35 -
 .../flink/table/plan/nodes/exec/ExecNode.scala     |   97 -
 .../table/plan/nodes/exec/StreamExecNode.scala     |   27 -
 .../plan/nodes/logical/FlinkLogicalAggregate.scala |  162 -
 .../plan/nodes/logical/FlinkLogicalCalc.scala      |   79 -
 .../plan/nodes/logical/FlinkLogicalCorrelate.scala |   99 -
 .../logical/FlinkLogicalDataStreamTableScan.scala  |  100 -
 .../plan/nodes/logical/FlinkLogicalExpand.scala    |   82 -
 .../FlinkLogicalIntermediateTableScan.scala        |   92 -
 .../plan/nodes/logical/FlinkLogicalIntersect.scala |   89 -
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |  108 -
 .../plan/nodes/logical/FlinkLogicalMatch.scala     |  132 -
 .../plan/nodes/logical/FlinkLogicalMinus.scala     |   87 -
 .../nodes/logical/FlinkLogicalOverAggregate.scala  |  106 -
 .../plan/nodes/logical/FlinkLogicalRank.scala      |  123 -
 .../table/plan/nodes/logical/FlinkLogicalRel.scala |   28 -
 .../plan/nodes/logical/FlinkLogicalSink.scala      |   80 -
 .../plan/nodes/logical/FlinkLogicalSnapshot.scala  |   90 -
 .../plan/nodes/logical/FlinkLogicalSort.scala      |  139 -
 .../logical/FlinkLogicalTableFunctionScan.scala    |  125 -
 .../logical/FlinkLogicalTableSourceScan.scala      |  124 -
 .../plan/nodes/logical/FlinkLogicalUnion.scala     |   93 -
 .../plan/nodes/logical/FlinkLogicalValues.scala    |   89 -
 .../logical/FlinkLogicalWatermarkAssigner.scala    |   79 -
 .../logical/FlinkLogicalWindowAggregate.scala      |  119 -
 .../plan/nodes/physical/FlinkPhysicalRel.scala     |   42 -
 .../nodes/physical/PhysicalTableSourceScan.scala   |   55 -
 .../batch/BatchExecBoundedStreamScan.scala         |  118 -
 .../plan/nodes/physical/batch/BatchExecCalc.scala  |  166 -
 .../nodes/physical/batch/BatchExecCorrelate.scala  |  207 --
 .../nodes/physical/batch/BatchExecExchange.scala   |  196 --
 .../nodes/physical/batch/BatchExecExpand.scala     |  111 -
 .../batch/BatchExecGroupAggregateBase.scala        |   94 -
 .../physical/batch/BatchExecHashAggregate.scala    |  155 -
 .../batch/BatchExecHashAggregateBase.scala         |  157 -
 .../nodes/physical/batch/BatchExecHashJoin.scala   |  285 --
 .../batch/BatchExecHashWindowAggregate.scala       |   98 -
 .../batch/BatchExecHashWindowAggregateBase.scala   |  159 -
 .../batch/BatchExecIntermediateTableScan.scala     |   44 -
 .../nodes/physical/batch/BatchExecJoinBase.scala   |  225 --
 .../plan/nodes/physical/batch/BatchExecLimit.scala |  121 -
 .../batch/BatchExecLocalHashAggregate.scala        |  132 -
 .../batch/BatchExecLocalHashWindowAggregate.scala  |   93 -
 .../batch/BatchExecLocalSortAggregate.scala        |  141 -
 .../batch/BatchExecLocalSortWindowAggregate.scala  |   91 -
 .../nodes/physical/batch/BatchExecLookupJoin.scala |  100 -
 .../physical/batch/BatchExecNestedLoopJoin.scala   |  184 --
 .../physical/batch/BatchExecOverAggregate.scala    |  631 ----
 .../plan/nodes/physical/batch/BatchExecRank.scala  |  304 --
 .../plan/nodes/physical/batch/BatchExecSink.scala  |  190 --
 .../plan/nodes/physical/batch/BatchExecSort.scala  |  138 -
 .../physical/batch/BatchExecSortAggregate.scala    |  170 --
 .../batch/BatchExecSortAggregateBase.scala         |  130 -
 .../nodes/physical/batch/BatchExecSortLimit.scala  |  158 -
 .../physical/batch/BatchExecSortMergeJoin.scala    |  287 --
 .../batch/BatchExecSortWindowAggregate.scala       |   98 -
 .../batch/BatchExecSortWindowAggregateBase.scala   |  145 -
 .../physical/batch/BatchExecTableSourceScan.scala  |  159 -
 .../plan/nodes/physical/batch/BatchExecUnion.scala |  119 -
 .../nodes/physical/batch/BatchExecValues.scala     |   88 -
 .../batch/BatchExecWindowAggregateBase.scala       |   83 -
 .../nodes/physical/batch/BatchPhysicalRel.scala    |   29 -
 .../nodes/physical/stream/StreamExecCalc.scala     |  132 -
 .../physical/stream/StreamExecCorrelate.scala      |  143 -
 .../physical/stream/StreamExecDataStreamScan.scala |  176 --
 .../physical/stream/StreamExecDeduplicate.scala    |  166 -
 .../nodes/physical/stream/StreamExecExchange.scala |  114 -
 .../nodes/physical/stream/StreamExecExpand.scala   |  110 -
 .../stream/StreamExecGlobalGroupAggregate.scala    |  227 --
 .../physical/stream/StreamExecGroupAggregate.scala |  220 --
 .../stream/StreamExecGroupAggregateBase.scala      |   54 -
 .../stream/StreamExecGroupWindowAggregate.scala    |  365 ---
 .../StreamExecIncrementalGroupAggregate.scala      |  221 --
 .../stream/StreamExecIntermediateTableScan.scala   |   61 -
 .../nodes/physical/stream/StreamExecJoin.scala     |  258 --
 .../nodes/physical/stream/StreamExecLimit.scala    |  183 --
 .../stream/StreamExecLocalGroupAggregate.scala     |  160 -
 .../physical/stream/StreamExecLookupJoin.scala     |  110 -
 .../nodes/physical/stream/StreamExecMatch.scala    |  463 ---
 .../physical/stream/StreamExecOverAggregate.scala  |  469 ---
 .../nodes/physical/stream/StreamExecRank.scala     |  254 --
 .../nodes/physical/stream/StreamExecSink.scala     |  239 --
 .../nodes/physical/stream/StreamExecSort.scala     |  154 -
 .../physical/stream/StreamExecSortLimit.scala      |  238 --
 .../stream/StreamExecTableSourceScan.scala         |  254 --
 .../physical/stream/StreamExecTemporalJoin.scala   |  426 ---
 .../physical/stream/StreamExecTemporalSort.scala   |  204 --
 .../nodes/physical/stream/StreamExecUnion.scala    |   90 -
 .../nodes/physical/stream/StreamExecValues.scala   |   92 -
 .../stream/StreamExecWatermarkAssigner.scala       |  185 --
 .../physical/stream/StreamExecWindowJoin.scala     |  360 ---
 .../nodes/physical/stream/StreamPhysicalRel.scala  |   55 -
 .../BatchCommonSubGraphBasedOptimizer.scala        |   87 -
 .../optimize/CommonSubGraphBasedOptimizer.scala    |  135 -
 .../flink/table/plan/optimize/Optimizer.scala      |   37 -
 .../flink/table/plan/optimize/RelNodeBlock.scala   |  444 ---
 .../StreamCommonSubGraphBasedOptimizer.scala       |  328 --
 .../optimize/program/BatchOptimizeContext.scala    |   26 -
 .../plan/optimize/program/FlinkBatchProgram.scala  |  211 --
 .../optimize/program/FlinkChainedProgram.scala     |  166 -
 .../optimize/program/FlinkDecorrelateProgram.scala |   97 -
 .../plan/optimize/program/FlinkGroupProgram.scala  |  107 -
 .../plan/optimize/program/FlinkHepProgram.scala    |  105 -
 .../optimize/program/FlinkHepRuleSetProgram.scala  |  196 --
 .../FlinkMiniBatchIntervalTraitInitProgram.scala   |   35 -
 .../optimize/program/FlinkOptimizeContext.scala    |   28 -
 .../optimize/program/FlinkOptimizeProgram.scala    |   36 -
 .../program/FlinkRelTimeIndicatorProgram.scala     |   36 -
 .../optimize/program/FlinkRuleSetProgram.scala     |   77 -
 .../plan/optimize/program/FlinkStreamProgram.scala |  208 --
 .../FlinkUpdateAsRetractionTraitInitProgram.scala  |   38 -
 .../optimize/program/FlinkVolcanoProgram.scala     |  132 -
 .../optimize/program/StreamOptimizeContext.scala   |   53 -
 .../plan/reuse/DeadlockBreakupProcessor.scala      |  345 ---
 .../flink/table/plan/reuse/SubplanReuser.scala     |  241 --
 .../table/plan/rules/FlinkBatchRuleSets.scala      |  404 ---
 .../table/plan/rules/FlinkStreamRuleSets.scala     |  415 ---
 .../logical/AggregateReduceGroupingRule.scala      |  124 -
 .../logical/BatchLogicalWindowAggregateRule.scala  |   80 -
 .../plan/rules/logical/CalcRankTransposeRule.scala |  197 --
 .../rules/logical/CalcSnapshotTransposeRule.scala  |   50 -
 .../rules/logical/ConvertToNotInOrInRule.scala     |  187 --
 .../rules/logical/DecomposeGroupingSetsRule.scala  |  374 ---
 .../logical/EnumerableToLogicalTableScan.scala     |   49 -
 .../plan/rules/logical/FlinkCalcMergeRule.scala    |  108 -
 .../plan/rules/logical/FlinkLimit0RemoveRule.scala |   50 -
 .../plan/rules/logical/FlinkLogicalRankRule.scala  |  270 --
 .../plan/rules/logical/FlinkPruneEmptyRules.scala  |   70 -
 .../rules/logical/FlinkRewriteSubQueryRule.scala   |  168 --
 .../rules/logical/FlinkSubQueryRemoveRule.scala    |  459 ---
 .../JoinConditionEqualityTransferRule.scala        |  172 --
 .../logical/JoinConditionTypeCoerceRule.scala      |  128 -
 .../JoinDependentConditionDerivationRule.scala     |  145 -
 .../rules/logical/JoinDeriveNullFilterRule.scala   |  107 -
 ...relateToJoinFromTemporalTableFunctionRule.scala |  230 --
 ...gicalCorrelateToJoinFromTemporalTableRule.scala |  129 -
 .../plan/rules/logical/LogicalUnnestRule.scala     |  181 --
 .../logical/LogicalWindowAggregateRuleBase.scala   |  212 --
 .../logical/ProjectSemiAntiJoinTransposeRule.scala |  167 -
 .../rules/logical/PruneAggregateCallRule.scala     |  201 --
 .../PushFilterIntoTableSourceScanRule.scala        |  151 -
 .../PushPartitionIntoTableSourceScanRule.scala     |  170 --
 .../PushProjectIntoTableSourceScanRule.scala       |  116 -
 .../rules/logical/RankNumberColumnRemoveRule.scala |   84 -
 .../logical/ReplaceIntersectWithSemiJoinRule.scala |   64 -
 .../logical/ReplaceMinusWithAntiJoinRule.scala     |   64 -
 .../plan/rules/logical/RewriteCoalesceRule.scala   |  237 --
 .../rules/logical/RewriteIntersectAllRule.scala    |  143 -
 .../plan/rules/logical/RewriteMinusAllRule.scala   |  121 -
 .../logical/RewriteMultiJoinConditionRule.scala    |  129 -
 .../logical/SimplifyFilterConditionRule.scala      |  103 -
 .../rules/logical/SimplifyJoinConditionRule.scala  |   70 -
 .../plan/rules/logical/SplitAggregateRule.scala    |  414 ---
 .../logical/StreamLogicalWindowAggregateRule.scala |   90 -
 .../WindowAggregateReduceFunctionsRule.scala       |   82 -
 .../rules/logical/WindowGroupReorderRule.scala     |  137 -
 .../plan/rules/logical/WindowPropertiesRule.scala  |  295 --
 .../rules/physical/FlinkExpandConversionRule.scala |  168 --
 .../physical/batch/BatchExecAggRuleBase.scala      |  173 --
 .../batch/BatchExecBoundedStreamScanRule.scala     |   58 -
 .../rules/physical/batch/BatchExecCalcRule.scala   |   55 -
 .../BatchExecConstantTableFunctionScanRule.scala   |   85 -
 .../physical/batch/BatchExecCorrelateRule.scala    |   85 -
 .../rules/physical/batch/BatchExecExpandRule.scala |   54 -
 .../physical/batch/BatchExecHashAggRule.scala      |  183 --
 .../physical/batch/BatchExecHashJoinRule.scala     |  196 --
 .../batch/BatchExecIntermediateTableScanRule.scala |   48 -
 .../physical/batch/BatchExecJoinRuleBase.scala     |  108 -
 .../rules/physical/batch/BatchExecLimitRule.scala  |  108 -
 .../physical/batch/BatchExecLookupJoinRule.scala   |  101 -
 .../batch/BatchExecNestedLoopJoinRule.scala        |   88 -
 .../batch/BatchExecNestedLoopJoinRuleBase.scala    |   65 -
 .../batch/BatchExecOverAggregateRule.scala         |  181 --
 .../rules/physical/batch/BatchExecRankRule.scala   |  119 -
 .../batch/BatchExecSingleRowJoinRule.scala         |   86 -
 .../rules/physical/batch/BatchExecSinkRule.scala   |   86 -
 .../physical/batch/BatchExecSortAggRule.scala      |  196 --
 .../physical/batch/BatchExecSortLimitRule.scala    |  109 -
 .../batch/BatchExecSortMergeJoinRule.scala         |  157 -
 .../rules/physical/batch/BatchExecSortRule.scala   |   90 -
 .../batch/BatchExecTableSourceScanRule.scala       |   68 -
 .../rules/physical/batch/BatchExecUnionRule.scala  |   60 -
 .../rules/physical/batch/BatchExecValuesRule.scala |   51 -
 .../batch/BatchExecWindowAggregateRule.scala       |  442 ---
 .../batch/RemoveRedundantLocalHashAggRule.scala    |   60 -
 .../batch/RemoveRedundantLocalRankRule.scala       |   60 -
 .../batch/RemoveRedundantLocalSortAggRule.scala    |  110 -
 .../physical/common/CommonLookupJoinRule.scala     |  152 -
 .../physical/stream/IncrementalAggregateRule.scala |  197 --
 .../stream/MiniBatchIntervalInferRule.scala        |  129 -
 .../rules/physical/stream/StreamExecCalcRule.scala |   55 -
 .../StreamExecConstantTableFunctionScanRule.scala  |   85 -
 .../physical/stream/StreamExecCorrelateRule.scala  |  129 -
 .../stream/StreamExecDataStreamScanRule.scala      |   61 -
 .../stream/StreamExecDeduplicateRule.scala         |  136 -
 .../physical/stream/StreamExecExpandRule.scala     |   55 -
 .../stream/StreamExecGroupAggregateRule.scala      |   81 -
 .../StreamExecGroupWindowAggregateRule.scala       |  102 -
 .../StreamExecIntermediateTableScanRule.scala      |   48 -
 .../rules/physical/stream/StreamExecJoinRule.scala |  140 -
 .../physical/stream/StreamExecLimitRule.scala      |   70 -
 .../physical/stream/StreamExecLookupJoinRule.scala |   98 -
 .../physical/stream/StreamExecMatchRule.scala      |  160 -
 .../stream/StreamExecOverAggregateRule.scala       |   79 -
 .../rules/physical/stream/StreamExecRankRule.scala |   75 -
 .../stream/StreamExecRetractionRules.scala         |  241 --
 .../rules/physical/stream/StreamExecSinkRule.scala |   85 -
 .../physical/stream/StreamExecSortLimitRule.scala  |   69 -
 .../rules/physical/stream/StreamExecSortRule.scala |   68 -
 .../stream/StreamExecTableSourceScanRule.scala     |   70 -
 .../stream/StreamExecTemporalJoinRule.scala        |  102 -
 .../stream/StreamExecTemporalSortRule.scala        |   91 -
 .../physical/stream/StreamExecUnionRule.scala      |   61 -
 .../physical/stream/StreamExecValuesRule.scala     |   53 -
 .../stream/StreamExecWatermarkAssignerRule.scala   |   64 -
 .../physical/stream/StreamExecWindowJoinRule.scala |  137 -
 .../stream/TwoStageOptimizedAggregateRule.scala    |  179 --
 .../flink/table/plan/schema/DataStreamTable.scala  |   77 -
 .../schema/DeferredTypeFlinkTableFunction.scala    |   64 -
 .../flink/table/plan/schema/FlinkRelOptTable.scala |  335 ---
 .../flink/table/plan/schema/FlinkTable.scala       |   52 -
 .../table/plan/schema/FlinkTableFunction.scala     |   61 -
 .../table/plan/schema/GenericRelDataType.scala     |   68 -
 .../flink/table/plan/schema/InlineTable.scala      |  124 -
 .../table/plan/schema/IntermediateRelTable.scala   |   64 -
 .../flink/table/plan/schema/TableSinkTable.scala   |   52 -
 .../table/plan/schema/TableSourceSinkTable.scala   |   75 -
 .../flink/table/plan/schema/TableSourceTable.scala |   80 -
 .../plan/schema/TimeIndicatorRelDataType.scala     |   54 -
 .../plan/schema/TypedFlinkTableFunction.scala      |   67 -
 .../flink/table/plan/stats/FlinkStatistic.scala    |  185 --
 .../flink/table/plan/stats/ValueInterval.scala     |  313 --
 .../table/plan/trait/FlinkRelDistribution.scala    |  258 --
 .../plan/trait/FlinkRelDistributionTraitDef.scala  |   48 -
 .../table/plan/trait/MiniBatchIntervalTrait.scala  |  100 -
 .../plan/trait/MiniBatchIntervalTraitDef.scala     |   48 -
 .../table/plan/trait/RelModifiedMonotonicity.scala |   42 -
 .../apache/flink/table/plan/trait/TraitUtil.scala  |   68 -
 .../table/plan/trait/retractionTraitDefs.scala     |   81 -
 .../flink/table/plan/trait/retractionTraits.scala  |  113 -
 .../flink/table/plan/util/AggFunctionFactory.scala |  636 ----
 .../flink/table/plan/util/AggregateUtil.scala      |  745 -----
 .../flink/table/plan/util/ColumnIntervalUtil.scala |  332 --
 .../flink/table/plan/util/CorrelateUtil.scala      |  137 -
 .../flink/table/plan/util/ExecNodePlanDumper.scala |  411 ---
 .../apache/flink/table/plan/util/ExpandUtil.scala  |  248 --
 .../table/plan/util/ExplodeFunctionUtil.scala      |  179 --
 .../flink/table/plan/util/FlinkRelMdUtil.scala     |  742 -----
 .../flink/table/plan/util/FlinkRelOptUtil.scala    |  606 ----
 .../flink/table/plan/util/FlinkRexUtil.scala       |  397 ---
 .../apache/flink/table/plan/util/JoinUtil.scala    |  140 -
 .../flink/table/plan/util/LookupJoinUtil.scala     |   47 -
 .../apache/flink/table/plan/util/MatchUtil.scala   |   54 -
 .../flink/table/plan/util/OverAggregateUtil.scala  |  113 -
 .../flink/table/plan/util/PartitionPruner.scala    |  169 --
 .../table/plan/util/RankProcessStrategy.scala      |  110 -
 .../apache/flink/table/plan/util/RankUtil.scala    |  308 --
 .../flink/table/plan/util/RelDigestUtil.scala      |  120 -
 .../flink/table/plan/util/RelExplainUtil.scala     |  872 ------
 .../apache/flink/table/plan/util/RelShuttles.scala |  216 --
 .../flink/table/plan/util/RelTreeWriterImpl.scala  |  129 -
 .../flink/table/plan/util/RexDefaultVisitor.scala  |   66 -
 .../flink/table/plan/util/RexNodeExtractor.scala   |  446 ---
 .../flink/table/plan/util/RexNodeRewriter.scala    |   62 -
 .../apache/flink/table/plan/util/ScanUtil.scala    |  134 -
 .../flink/table/plan/util/SetOpRewriteUtil.scala   |  118 -
 .../apache/flink/table/plan/util/SortUtil.scala    |  129 -
 .../flink/table/plan/util/TemporalJoinUtil.scala   |  105 -
 .../table/plan/util/UpdatingPlanChecker.scala      |  247 --
 .../flink/table/plan/util/WindowEmitStrategy.scala |  210 --
 .../flink/table/plan/util/WindowJoinUtil.scala     |  522 ----
 .../apache/flink/table/plan/util/aggregation.scala |  136 -
 .../apache/flink/table/planner/BatchPlanner.scala  |  120 -
 .../apache/flink/table/planner/PlannerBase.scala   |  293 --
 .../apache/flink/table/planner/StreamPlanner.scala |  121 -
 .../table/planner/calcite/CalciteConfig.scala      |  228 ++
 .../planner/calcite/FlinkCalciteSqlValidator.scala |   95 +
 .../flink/table/planner/calcite/FlinkContext.scala |   45 +
 .../table/planner/calcite/FlinkContextImpl.scala   |   32 +
 .../table/planner/calcite/FlinkLocalRef.scala      |   49 +
 .../planner/calcite/FlinkLogicalRelFactories.scala |  262 ++
 .../table/planner/calcite/FlinkPlannerImpl.scala   |  210 ++
 .../table/planner/calcite/FlinkRelBuilder.scala    |  169 ++
 .../table/planner/calcite/FlinkRelFactories.scala  |  124 +
 .../table/planner/calcite/FlinkRelOptCluster.scala |   77 +
 .../calcite/FlinkRelOptClusterFactory.scala        |   43 +
 .../table/planner/calcite/FlinkTypeFactory.scala   |  518 ++++
 .../table/planner/calcite/FlinkTypeSystem.scala    |  148 +
 .../planner/calcite/PreValidateReWriter.scala      |  201 ++
 .../calcite/RelTimeIndicatorConverter.scala        |  701 +++++
 .../table/planner/codegen/CalcCodeGenerator.scala  |  215 ++
 .../table/planner/codegen/CodeGenException.scala   |   24 +
 .../flink/table/planner/codegen/CodeGenUtils.scala |  721 +++++
 .../planner/codegen/CodeGeneratorContext.scala     |  830 +++++
 .../planner/codegen/CollectorCodeGenerator.scala   |   98 +
 .../planner/codegen/CorrelateCodeGenerator.scala   |  427 +++
 .../planner/codegen/EqualiserCodeGenerator.scala   |  147 +
 .../planner/codegen/ExpandCodeGenerator.scala      |   77 +
 .../table/planner/codegen/ExprCodeGenerator.scala  |  757 +++++
 .../table/planner/codegen/ExpressionReducer.scala  |  255 ++
 .../planner/codegen/FunctionCodeGenerator.scala    |  218 ++
 .../table/planner/codegen/GenerateUtils.scala      |  827 +++++
 .../planner/codegen/GeneratedExpression.scala      |  103 +
 .../table/planner/codegen/HashCodeGenerator.scala  |  106 +
 .../flink/table/planner/codegen/Indenter.scala     |   58 +
 .../planner/codegen/InputFormatCodeGenerator.scala |   93 +
 .../planner/codegen/LongHashJoinGenerator.scala    |  359 +++
 .../planner/codegen/LookupJoinCodeGenerator.scala  |  456 +++
 .../table/planner/codegen/MatchCodeGenerator.scala |  815 +++++
 .../codegen/NestedLoopJoinCodeGenerator.scala      |  370 +++
 .../planner/codegen/OperatorCodeGenerator.scala    |  232 ++
 .../planner/codegen/ProjectionCodeGenerator.scala  |  214 ++
 .../table/planner/codegen/SinkCodeGenerator.scala  |  276 ++
 .../planner/codegen/ValuesCodeGenerator.scala      |   60 +
 .../table/planner/codegen/agg/AggCodeGen.scala     |   49 +
 .../codegen/agg/AggsHandlerCodeGenerator.scala     |  825 +++++
 .../codegen/agg/DeclarativeAggCodeGen.scala        |  310 ++
 .../planner/codegen/agg/DistinctAggCodeGen.scala   |  935 ++++++
 .../planner/codegen/agg/ImperativeAggCodeGen.scala |  507 ++++
 .../codegen/agg/batch/AggCodeGenHelper.scala       |  717 +++++
 .../agg/batch/AggWithoutKeysCodeGenerator.scala    |  118 +
 .../codegen/agg/batch/HashAggCodeGenHelper.scala   |  847 ++++++
 .../codegen/agg/batch/HashAggCodeGenerator.scala   |  230 ++
 .../agg/batch/HashWindowCodeGenerator.scala        |  815 +++++
 .../codegen/agg/batch/SortAggCodeGenerator.scala   |  149 +
 .../agg/batch/SortWindowCodeGenerator.scala        |  269 ++
 .../codegen/agg/batch/WindowCodeGenerator.scala    |  765 +++++
 .../planner/codegen/calls/BuiltInMethods.scala     |  494 +++
 .../planner/codegen/calls/CallGenerator.scala      |   35 +
 .../planner/codegen/calls/ConstantCallGen.scala    |   37 +
 .../codegen/calls/CurrentTimePointCallGen.scala    |   58 +
 .../table/planner/codegen/calls/DivCallGen.scala   |   67 +
 .../planner/codegen/calls/ExtractCallGen.scala     |  115 +
 .../planner/codegen/calls/FloorCeilCallGen.scala   |   93 +
 .../planner/codegen/calls/FunctionGenerator.scala  |  923 ++++++
 .../planner/codegen/calls/HashCodeCallGen.scala    |   36 +
 .../table/planner/codegen/calls/IfCallGen.scala    |   67 +
 .../table/planner/codegen/calls/LikeCallGen.scala  |  172 ++
 .../planner/codegen/calls/MethodCallGen.scala      |   70 +
 .../table/planner/codegen/calls/PrintCallGen.scala |   67 +
 .../table/planner/codegen/calls/RandCallGen.scala  |   56 +
 .../codegen/calls/ScalarFunctionCallGen.scala      |  138 +
 .../planner/codegen/calls/ScalarOperatorGens.scala | 2177 ++++++++++++++
 .../planner/codegen/calls/StringCallGen.scala      |  813 +++++
 .../codegen/calls/TableFunctionCallGen.scala       |   93 +
 .../codegen/calls/TimestampDiffCallGen.scala       |  120 +
 ...ltiFieldRangeBoundComparatorCodeGenerator.scala |   84 +
 .../over/RangeBoundComparatorCodeGenerator.scala   |  177 ++
 .../codegen/sort/ComparatorCodeGenerator.scala     |   84 +
 .../planner/codegen/sort/SortCodeGenerator.scala   |  464 +++
 .../table/planner/dataview/DataViewSpec.scala      |   48 +
 .../table/planner/dataview/DataViewUtils.scala     |  215 ++
 .../table/planner/delegation/BatchPlanner.scala    |  119 +
 .../table/planner/delegation/PlannerBase.scala     |  294 ++
 .../table/planner/delegation/StreamPlanner.scala   |  120 +
 .../planner/expressions/ExpressionBridge.scala     |   41 +
 .../table/planner/expressions/InputTypeSpec.scala  |   69 +
 .../planner/expressions/PlannerExpression.scala    |   88 +
 .../expressions/PlannerExpressionConverter.scala   |  849 ++++++
 .../expressions/PlannerExpressionUtils.scala       |   68 +
 .../table/planner/expressions/Reinterpret.scala    |   45 +
 .../planner/expressions/ReturnTypeInference.scala  |  220 ++
 .../table/planner/expressions/aggregations.scala   |  230 ++
 .../table/planner/expressions/arithmetic.scala     |  149 +
 .../flink/table/planner/expressions/call.scala     |  223 ++
 .../flink/table/planner/expressions/cast.scala     |   44 +
 .../table/planner/expressions/collection.scala     |  174 ++
 .../table/planner/expressions/comparison.scala     |  181 ++
 .../table/planner/expressions/composite.scala      |  100 +
 .../planner/expressions/fieldExpression.scala      |  230 ++
 .../planner/expressions/hashExpressions.scala      |   93 +
 .../flink/table/planner/expressions/literals.scala |   82 +
 .../flink/table/planner/expressions/logic.scala    |   87 +
 .../planner/expressions/mathExpressions.scala      |  401 +++
 .../flink/table/planner/expressions/ordering.scala |   43 +
 .../table/planner/expressions/overOffsets.scala    |   54 +
 .../table/{ => planner}/expressions/package.scala  |    0
 .../expressions/plannerWindowProperties.scala      |   86 +
 .../planner/expressions/stringExpressions.scala    |  490 +++
 .../flink/table/planner/expressions/subquery.scala |   76 +
 .../flink/table/planner/expressions/symbols.scala  |  127 +
 .../flink/table/planner/expressions/time.scala     |  322 ++
 .../planner/expressions/windowProperties.scala     |   61 +
 .../planner/functions/utils/AggSqlFunction.scala   |  216 ++
 .../functions/utils/ScalarSqlFunction.scala        |  199 ++
 .../planner/functions/utils/TableSqlFunction.scala |  191 ++
 .../functions/utils/UserDefinedFunctionUtils.scala |  858 ++++++
 .../org/apache/flink/table/planner/package.scala   |   43 +
 .../apache/flink/table/planner/plan/TreeNode.scala |  115 +
 .../flink/table/planner/plan/cost/FlinkCost.scala  |  354 +++
 .../table/planner/plan/cost/FlinkCostBase.scala    |   37 +
 .../table/planner/plan/cost/FlinkCostFactory.scala |   49 +
 .../planner/plan/cost/FlinkCostFactoryBase.scala   |   35 +
 .../planner/plan/logical/MatchRecognize.scala      |   42 +
 .../table/planner/plan/logical/groupWindows.scala  |   79 +
 .../metadata/AggCallSelectivityEstimator.scala     |  455 +++
 .../metadata/FlinkDefaultRelMetadataProvider.scala |   53 +
 .../plan/metadata/FlinkRelMdColumnInterval.scala   |  746 +++++
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |  295 ++
 .../metadata/FlinkRelMdColumnOriginNullCount.scala |  134 +
 .../plan/metadata/FlinkRelMdColumnUniqueness.scala |  739 +++++
 .../plan/metadata/FlinkRelMdCumulativeCost.scala   |   59 +
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |  626 ++++
 .../plan/metadata/FlinkRelMdDistribution.scala     |  103 +
 .../FlinkRelMdFilteredColumnInterval.scala         |  264 ++
 .../metadata/FlinkRelMdModifiedMonotonicity.scala  |  552 ++++
 .../metadata/FlinkRelMdNonCumulativeCost.scala     |   55 +
 .../FlinkRelMdPercentageOriginalRows.scala         |  146 +
 .../plan/metadata/FlinkRelMdPopulationSize.scala   |  412 +++
 .../planner/plan/metadata/FlinkRelMdRowCount.scala |  456 +++
 .../plan/metadata/FlinkRelMdSelectivity.scala      |  301 ++
 .../planner/plan/metadata/FlinkRelMdSize.scala     |  436 +++
 .../plan/metadata/FlinkRelMdUniqueGroups.scala     |  432 +++
 .../plan/metadata/FlinkRelMdUniqueKeys.scala       |  566 ++++
 .../plan/metadata/SelectivityEstimator.scala       | 1146 +++++++
 .../planner/plan/nodes/FlinkConventions.scala      |   55 +
 .../table/planner/plan/nodes/FlinkRelNode.scala    |  118 +
 .../table/planner/plan/nodes/calcite/Expand.scala  |  109 +
 .../planner/plan/nodes/calcite/LogicalExpand.scala |   58 +
 .../planner/plan/nodes/calcite/LogicalRank.scala   |   96 +
 .../planner/plan/nodes/calcite/LogicalSink.scala   |   57 +
 .../nodes/calcite/LogicalWatermarkAssigner.scala   |   43 +
 .../nodes/calcite/LogicalWindowAggregate.scala     |   89 +
 .../table/planner/plan/nodes/calcite/Rank.scala    |  137 +
 .../table/planner/plan/nodes/calcite/Sink.scala    |   58 +
 .../plan/nodes/calcite/WatermarkAssigner.scala     |   65 +
 .../plan/nodes/calcite/WindowAggregate.scala       |   87 +
 .../planner/plan/nodes/common/CommonCalc.scala     |   87 +
 .../nodes/common/CommonIntermediateTableScan.scala |   54 +
 .../plan/nodes/common/CommonLookupJoin.scala       |  678 +++++
 .../plan/nodes/common/CommonPhysicalExchange.scala |  109 +
 .../plan/nodes/common/CommonPhysicalJoin.scala     |   86 +
 .../planner/plan/nodes/exec/BatchExecNode.scala    |   35 +
 .../table/planner/plan/nodes/exec/ExecNode.scala   |   97 +
 .../planner/plan/nodes/exec/StreamExecNode.scala   |   27 +
 .../plan/nodes/logical/FlinkLogicalAggregate.scala |  162 +
 .../plan/nodes/logical/FlinkLogicalCalc.scala      |   79 +
 .../plan/nodes/logical/FlinkLogicalCorrelate.scala |   99 +
 .../logical/FlinkLogicalDataStreamTableScan.scala  |  100 +
 .../plan/nodes/logical/FlinkLogicalExpand.scala    |   82 +
 .../FlinkLogicalIntermediateTableScan.scala        |   92 +
 .../plan/nodes/logical/FlinkLogicalIntersect.scala |   89 +
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |  108 +
 .../plan/nodes/logical/FlinkLogicalMatch.scala     |  133 +
 .../plan/nodes/logical/FlinkLogicalMinus.scala     |   87 +
 .../nodes/logical/FlinkLogicalOverAggregate.scala  |  106 +
 .../plan/nodes/logical/FlinkLogicalRank.scala      |  123 +
 .../plan/nodes/logical/FlinkLogicalRel.scala       |   28 +
 .../plan/nodes/logical/FlinkLogicalSink.scala      |   80 +
 .../plan/nodes/logical/FlinkLogicalSnapshot.scala  |   90 +
 .../plan/nodes/logical/FlinkLogicalSort.scala      |  139 +
 .../logical/FlinkLogicalTableFunctionScan.scala    |  125 +
 .../logical/FlinkLogicalTableSourceScan.scala      |  124 +
 .../plan/nodes/logical/FlinkLogicalUnion.scala     |   93 +
 .../plan/nodes/logical/FlinkLogicalValues.scala    |   89 +
 .../logical/FlinkLogicalWatermarkAssigner.scala    |   79 +
 .../logical/FlinkLogicalWindowAggregate.scala      |  119 +
 .../plan/nodes/physical/FlinkPhysicalRel.scala     |   42 +
 .../nodes/physical/PhysicalTableSourceScan.scala   |   55 +
 .../batch/BatchExecBoundedStreamScan.scala         |  118 +
 .../plan/nodes/physical/batch/BatchExecCalc.scala  |  166 +
 .../nodes/physical/batch/BatchExecCorrelate.scala  |  207 ++
 .../nodes/physical/batch/BatchExecExchange.scala   |  196 ++
 .../nodes/physical/batch/BatchExecExpand.scala     |  111 +
 .../batch/BatchExecGroupAggregateBase.scala        |   94 +
 .../physical/batch/BatchExecHashAggregate.scala    |  155 +
 .../batch/BatchExecHashAggregateBase.scala         |  157 +
 .../nodes/physical/batch/BatchExecHashJoin.scala   |  285 ++
 .../batch/BatchExecHashWindowAggregate.scala       |   98 +
 .../batch/BatchExecHashWindowAggregateBase.scala   |  159 +
 .../batch/BatchExecIntermediateTableScan.scala     |   45 +
 .../nodes/physical/batch/BatchExecJoinBase.scala   |  225 ++
 .../plan/nodes/physical/batch/BatchExecLimit.scala |  121 +
 .../batch/BatchExecLocalHashAggregate.scala        |  132 +
 .../batch/BatchExecLocalHashWindowAggregate.scala  |   93 +
 .../batch/BatchExecLocalSortAggregate.scala        |  141 +
 .../batch/BatchExecLocalSortWindowAggregate.scala  |   91 +
 .../nodes/physical/batch/BatchExecLookupJoin.scala |  100 +
 .../physical/batch/BatchExecNestedLoopJoin.scala   |  184 ++
 .../physical/batch/BatchExecOverAggregate.scala    |  631 ++++
 .../plan/nodes/physical/batch/BatchExecRank.scala  |  304 ++
 .../plan/nodes/physical/batch/BatchExecSink.scala  |  191 ++
 .../plan/nodes/physical/batch/BatchExecSort.scala  |  138 +
 .../physical/batch/BatchExecSortAggregate.scala    |  170 ++
 .../batch/BatchExecSortAggregateBase.scala         |  130 +
 .../nodes/physical/batch/BatchExecSortLimit.scala  |  158 +
 .../physical/batch/BatchExecSortMergeJoin.scala    |  287 ++
 .../batch/BatchExecSortWindowAggregate.scala       |   96 +
 .../batch/BatchExecSortWindowAggregateBase.scala   |  145 +
 .../physical/batch/BatchExecTableSourceScan.scala  |  159 +
 .../plan/nodes/physical/batch/BatchExecUnion.scala |  119 +
 .../nodes/physical/batch/BatchExecValues.scala     |   88 +
 .../batch/BatchExecWindowAggregateBase.scala       |   83 +
 .../nodes/physical/batch/BatchPhysicalRel.scala    |   29 +
 .../nodes/physical/stream/StreamExecCalc.scala     |  132 +
 .../physical/stream/StreamExecCorrelate.scala      |  143 +
 .../physical/stream/StreamExecDataStreamScan.scala |  176 ++
 .../physical/stream/StreamExecDeduplicate.scala    |  166 +
 .../nodes/physical/stream/StreamExecExchange.scala |  114 +
 .../nodes/physical/stream/StreamExecExpand.scala   |  110 +
 .../stream/StreamExecGlobalGroupAggregate.scala    |  227 ++
 .../physical/stream/StreamExecGroupAggregate.scala |  220 ++
 .../stream/StreamExecGroupAggregateBase.scala      |   54 +
 .../stream/StreamExecGroupWindowAggregate.scala    |  365 +++
 .../StreamExecIncrementalGroupAggregate.scala      |  221 ++
 .../stream/StreamExecIntermediateTableScan.scala   |   62 +
 .../nodes/physical/stream/StreamExecJoin.scala     |  258 ++
 .../nodes/physical/stream/StreamExecLimit.scala    |  183 ++
 .../stream/StreamExecLocalGroupAggregate.scala     |  160 +
 .../physical/stream/StreamExecLookupJoin.scala     |  110 +
 .../nodes/physical/stream/StreamExecMatch.scala    |  463 +++
 .../physical/stream/StreamExecOverAggregate.scala  |  469 +++
 .../nodes/physical/stream/StreamExecRank.scala     |  254 ++
 .../nodes/physical/stream/StreamExecSink.scala     |  240 ++
 .../nodes/physical/stream/StreamExecSort.scala     |  154 +
 .../physical/stream/StreamExecSortLimit.scala      |  238 ++
 .../stream/StreamExecTableSourceScan.scala         |  254 ++
 .../physical/stream/StreamExecTemporalJoin.scala   |  426 +++
 .../physical/stream/StreamExecTemporalSort.scala   |  204 ++
 .../nodes/physical/stream/StreamExecUnion.scala    |   90 +
 .../nodes/physical/stream/StreamExecValues.scala   |   92 +
 .../stream/StreamExecWatermarkAssigner.scala       |  185 ++
 .../physical/stream/StreamExecWindowJoin.scala     |  360 +++
 .../nodes/physical/stream/StreamPhysicalRel.scala  |   55 +
 .../BatchCommonSubGraphBasedOptimizer.scala        |   87 +
 .../optimize/CommonSubGraphBasedOptimizer.scala    |  135 +
 .../table/planner/plan/optimize/Optimizer.scala    |   37 +
 .../table/planner/plan/optimize/RelNodeBlock.scala |  444 +++
 .../StreamCommonSubGraphBasedOptimizer.scala       |  329 ++
 .../optimize/program/BatchOptimizeContext.scala    |   26 +
 .../plan/optimize/program/FlinkBatchProgram.scala  |  211 ++
 .../optimize/program/FlinkChainedProgram.scala     |  166 +
 .../optimize/program/FlinkDecorrelateProgram.scala |   97 +
 .../plan/optimize/program/FlinkGroupProgram.scala  |  107 +
 .../plan/optimize/program/FlinkHepProgram.scala    |  105 +
 .../optimize/program/FlinkHepRuleSetProgram.scala  |  196 ++
 .../FlinkMiniBatchIntervalTraitInitProgram.scala   |   35 +
 .../optimize/program/FlinkOptimizeContext.scala    |   28 +
 .../optimize/program/FlinkOptimizeProgram.scala    |   36 +
 .../program/FlinkRelTimeIndicatorProgram.scala     |   36 +
 .../optimize/program/FlinkRuleSetProgram.scala     |   77 +
 .../plan/optimize/program/FlinkStreamProgram.scala |  208 ++
 .../FlinkUpdateAsRetractionTraitInitProgram.scala  |   38 +
 .../optimize/program/FlinkVolcanoProgram.scala     |  132 +
 .../optimize/program/StreamOptimizeContext.scala   |   53 +
 .../plan/reuse/DeadlockBreakupProcessor.scala      |  345 +++
 .../table/planner/plan/reuse/SubplanReuser.scala   |  241 ++
 .../planner/plan/rules/FlinkBatchRuleSets.scala    |  404 +++
 .../planner/plan/rules/FlinkStreamRuleSets.scala   |  415 +++
 .../logical/AggregateReduceGroupingRule.scala      |  124 +
 .../logical/BatchLogicalWindowAggregateRule.scala  |   80 +
 .../plan/rules/logical/CalcRankTransposeRule.scala |  197 ++
 .../rules/logical/CalcSnapshotTransposeRule.scala  |   51 +
 .../rules/logical/ConvertToNotInOrInRule.scala     |  187 ++
 .../rules/logical/DecomposeGroupingSetsRule.scala  |  374 +++
 .../logical/EnumerableToLogicalTableScan.scala     |   49 +
 .../plan/rules/logical/FlinkCalcMergeRule.scala    |  108 +
 .../plan/rules/logical/FlinkLimit0RemoveRule.scala |   50 +
 .../plan/rules/logical/FlinkLogicalRankRule.scala  |  270 ++
 .../plan/rules/logical/FlinkPruneEmptyRules.scala  |   70 +
 .../rules/logical/FlinkRewriteSubQueryRule.scala   |  168 ++
 .../rules/logical/FlinkSubQueryRemoveRule.scala    |  459 +++
 .../JoinConditionEqualityTransferRule.scala        |  172 ++
 .../logical/JoinConditionTypeCoerceRule.scala      |  128 +
 .../JoinDependentConditionDerivationRule.scala     |  145 +
 .../rules/logical/JoinDeriveNullFilterRule.scala   |  107 +
 ...relateToJoinFromTemporalTableFunctionRule.scala |  231 ++
 ...gicalCorrelateToJoinFromTemporalTableRule.scala |  130 +
 .../plan/rules/logical/LogicalUnnestRule.scala     |  181 ++
 .../logical/LogicalWindowAggregateRuleBase.scala   |  213 ++
 .../logical/ProjectSemiAntiJoinTransposeRule.scala |  167 +
 .../rules/logical/PruneAggregateCallRule.scala     |  201 ++
 .../PushFilterIntoTableSourceScanRule.scala        |  152 +
 .../PushPartitionIntoTableSourceScanRule.scala     |  170 ++
 .../PushProjectIntoTableSourceScanRule.scala       |  116 +
 .../rules/logical/RankNumberColumnRemoveRule.scala |   84 +
 .../logical/ReplaceIntersectWithSemiJoinRule.scala |   64 +
 .../logical/ReplaceMinusWithAntiJoinRule.scala     |   64 +
 .../plan/rules/logical/RewriteCoalesceRule.scala   |  237 ++
 .../rules/logical/RewriteIntersectAllRule.scala    |  143 +
 .../plan/rules/logical/RewriteMinusAllRule.scala   |  121 +
 .../logical/RewriteMultiJoinConditionRule.scala    |  129 +
 .../logical/SimplifyFilterConditionRule.scala      |  103 +
 .../rules/logical/SimplifyJoinConditionRule.scala  |   70 +
 .../plan/rules/logical/SplitAggregateRule.scala    |  414 +++
 .../logical/StreamLogicalWindowAggregateRule.scala |   90 +
 .../WindowAggregateReduceFunctionsRule.scala       |   82 +
 .../rules/logical/WindowGroupReorderRule.scala     |  137 +
 .../plan/rules/logical/WindowPropertiesRule.scala  |  296 ++
 .../rules/physical/FlinkExpandConversionRule.scala |  168 ++
 .../physical/batch/BatchExecAggRuleBase.scala      |  173 ++
 .../batch/BatchExecBoundedStreamScanRule.scala     |   58 +
 .../rules/physical/batch/BatchExecCalcRule.scala   |   55 +
 .../BatchExecConstantTableFunctionScanRule.scala   |   85 +
 .../physical/batch/BatchExecCorrelateRule.scala    |   86 +
 .../rules/physical/batch/BatchExecExpandRule.scala |   55 +
 .../physical/batch/BatchExecHashAggRule.scala      |  183 ++
 .../physical/batch/BatchExecHashJoinRule.scala     |  196 ++
 .../batch/BatchExecIntermediateTableScanRule.scala |   48 +
 .../physical/batch/BatchExecJoinRuleBase.scala     |  107 +
 .../rules/physical/batch/BatchExecLimitRule.scala  |  108 +
 .../physical/batch/BatchExecLookupJoinRule.scala   |  101 +
 .../batch/BatchExecNestedLoopJoinRule.scala        |   88 +
 .../batch/BatchExecNestedLoopJoinRuleBase.scala    |   65 +
 .../batch/BatchExecOverAggregateRule.scala         |  181 ++
 .../rules/physical/batch/BatchExecRankRule.scala   |  119 +
 .../batch/BatchExecSingleRowJoinRule.scala         |   86 +
 .../rules/physical/batch/BatchExecSinkRule.scala   |   88 +
 .../physical/batch/BatchExecSortAggRule.scala      |  196 ++
 .../physical/batch/BatchExecSortLimitRule.scala    |  109 +
 .../batch/BatchExecSortMergeJoinRule.scala         |  157 +
 .../rules/physical/batch/BatchExecSortRule.scala   |   90 +
 .../batch/BatchExecTableSourceScanRule.scala       |   69 +
 .../rules/physical/batch/BatchExecUnionRule.scala  |   61 +
 .../rules/physical/batch/BatchExecValuesRule.scala |   52 +
 .../batch/BatchExecWindowAggregateRule.scala       |  442 +++
 .../batch/RemoveRedundantLocalHashAggRule.scala    |   60 +
 .../batch/RemoveRedundantLocalRankRule.scala       |   60 +
 .../batch/RemoveRedundantLocalSortAggRule.scala    |  110 +
 .../physical/common/CommonLookupJoinRule.scala     |  153 +
 .../physical/stream/IncrementalAggregateRule.scala |  197 ++
 .../stream/MiniBatchIntervalInferRule.scala        |  129 +
 .../rules/physical/stream/StreamExecCalcRule.scala |   55 +
 .../StreamExecConstantTableFunctionScanRule.scala  |   85 +
 .../physical/stream/StreamExecCorrelateRule.scala  |  129 +
 .../stream/StreamExecDataStreamScanRule.scala      |   61 +
 .../stream/StreamExecDeduplicateRule.scala         |  136 +
 .../physical/stream/StreamExecExpandRule.scala     |   55 +
 .../stream/StreamExecGroupAggregateRule.scala      |   81 +
 .../StreamExecGroupWindowAggregateRule.scala       |  102 +
 .../StreamExecIntermediateTableScanRule.scala      |   48 +
 .../rules/physical/stream/StreamExecJoinRule.scala |  141 +
 .../physical/stream/StreamExecLimitRule.scala      |   70 +
 .../physical/stream/StreamExecLookupJoinRule.scala |   98 +
 .../physical/stream/StreamExecMatchRule.scala      |  161 +
 .../stream/StreamExecOverAggregateRule.scala       |   79 +
 .../rules/physical/stream/StreamExecRankRule.scala |   75 +
 .../stream/StreamExecRetractionRules.scala         |  241 ++
 .../rules/physical/stream/StreamExecSinkRule.scala |   86 +
 .../physical/stream/StreamExecSortLimitRule.scala  |   69 +
 .../rules/physical/stream/StreamExecSortRule.scala |   67 +
 .../stream/StreamExecTableSourceScanRule.scala     |   70 +
 .../stream/StreamExecTemporalJoinRule.scala        |  102 +
 .../stream/StreamExecTemporalSortRule.scala        |   91 +
 .../physical/stream/StreamExecUnionRule.scala      |   61 +
 .../physical/stream/StreamExecValuesRule.scala     |   53 +
 .../stream/StreamExecWatermarkAssignerRule.scala   |   64 +
 .../physical/stream/StreamExecWindowJoinRule.scala |  136 +
 .../stream/TwoStageOptimizedAggregateRule.scala    |  179 ++
 .../planner/plan/schema/DataStreamTable.scala      |   77 +
 .../schema/DeferredTypeFlinkTableFunction.scala    |   64 +
 .../planner/plan/schema/FlinkRelOptTable.scala     |  335 +++
 .../table/planner/plan/schema/FlinkTable.scala     |   53 +
 .../planner/plan/schema/FlinkTableFunction.scala   |   61 +
 .../planner/plan/schema/GenericRelDataType.scala   |   69 +
 .../table/planner/plan/schema/InlineTable.scala    |  124 +
 .../planner/plan/schema/IntermediateRelTable.scala |   64 +
 .../table/planner/plan/schema/TableSinkTable.scala |   52 +
 .../planner/plan/schema/TableSourceSinkTable.scala |   75 +
 .../planner/plan/schema/TableSourceTable.scala     |   81 +
 .../plan/schema/TimeIndicatorRelDataType.scala     |   54 +
 .../plan/schema/TypedFlinkTableFunction.scala      |   67 +
 .../table/planner/plan/stats/FlinkStatistic.scala  |  186 ++
 .../table/planner/plan/stats/ValueInterval.scala   |  313 ++
 .../planner/plan/trait/FlinkRelDistribution.scala  |  258 ++
 .../plan/trait/FlinkRelDistributionTraitDef.scala  |   48 +
 .../plan/trait/MiniBatchIntervalTrait.scala        |  100 +
 .../plan/trait/MiniBatchIntervalTraitDef.scala     |   48 +
 .../plan/trait/RelModifiedMonotonicity.scala       |   42 +
 .../flink/table/planner/plan/trait/TraitUtil.scala |   68 +
 .../planner/plan/trait/retractionTraitDefs.scala   |   81 +
 .../planner/plan/trait/retractionTraits.scala      |  113 +
 .../planner/plan/utils/AggFunctionFactory.scala    |  636 ++++
 .../table/planner/plan/utils/AggregateUtil.scala   |  747 +++++
 .../planner/plan/utils/ColumnIntervalUtil.scala    |  332 ++
 .../table/planner/plan/utils/CorrelateUtil.scala   |  137 +
 .../planner/plan/utils/ExecNodePlanDumper.scala    |  411 +++
 .../table/planner/plan/utils/ExpandUtil.scala      |  248 ++
 .../planner/plan/utils/ExplodeFunctionUtil.scala   |  178 ++
 .../table/planner/plan/utils/FlinkRelMdUtil.scala  |  742 +++++
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |  606 ++++
 .../table/planner/plan/utils/FlinkRexUtil.scala    |  397 +++
 .../flink/table/planner/plan/utils/JoinUtil.scala  |  141 +
 .../table/planner/plan/utils/LookupJoinUtil.scala  |   48 +
 .../flink/table/planner/plan/utils/MatchUtil.scala |   55 +
 .../planner/plan/utils/OverAggregateUtil.scala     |  114 +
 .../table/planner/plan/utils/PartitionPruner.scala |  169 ++
 .../planner/plan/utils/RankProcessStrategy.scala   |  110 +
 .../flink/table/planner/plan/utils/RankUtil.scala  |  308 ++
 .../table/planner/plan/utils/RelDigestUtil.scala   |  120 +
 .../table/planner/plan/utils/RelExplainUtil.scala  |  873 ++++++
 .../table/planner/plan/utils/RelShuttles.scala     |  216 ++
 .../planner/plan/utils/RelTreeWriterImpl.scala     |  129 +
 .../planner/plan/utils/RexDefaultVisitor.scala     |   66 +
 .../planner/plan/utils/RexNodeExtractor.scala      |  445 +++
 .../table/planner/plan/utils/RexNodeRewriter.scala |   62 +
 .../flink/table/planner/plan/utils/ScanUtil.scala  |  134 +
 .../planner/plan/utils/SetOpRewriteUtil.scala      |  118 +
 .../flink/table/planner/plan/utils/SortUtil.scala  |  129 +
 .../planner/plan/utils/TemporalJoinUtil.scala      |  106 +
 .../planner/plan/utils/UpdatingPlanChecker.scala   |  247 ++
 .../planner/plan/utils/WindowEmitStrategy.scala    |  210 ++
 .../table/planner/plan/utils/WindowJoinUtil.scala  |  522 ++++
 .../table/planner/plan/utils/aggregation.scala     |  136 +
 .../table/planner/sinks/CollectTableSink.scala     |   84 +
 .../table/planner/sinks/DataStreamTableSink.scala  |   69 +
 .../flink/table/planner/sinks/TableSinkUtils.scala |  107 +
 .../table/planner/sources/TableSourceUtil.scala    |  388 +++
 .../table/planner/typeutils/TypeCoercion.scala     |  166 +
 .../typeutils/TypeInfoCheckUtils.scala             |    0
 .../planner/utils/JavaScalaConversionUtil.scala    |   71 +
 .../apache/flink/table/planner/utils/Logging.scala |   28 +
 .../flink/table/planner/utils/PlanUtil.scala       |   81 +
 .../{ => planner}/validate/ValidationResult.scala  |    0
 .../flink/table/sinks/CollectTableSink.scala       |   83 -
 .../flink/table/sinks/DataStreamTableSink.scala    |   68 -
 .../apache/flink/table/sinks/TableSinkUtils.scala  |  106 -
 .../flink/table/sources/TableSourceUtil.scala      |  387 ---
 .../flink/table/typeutils/TypeCoercion.scala       |  166 -
 .../flink/table/util/JavaScalaConversionUtil.scala |   71 -
 .../org/apache/flink/table/util/Logging.scala      |   28 -
 .../org/apache/flink/table/util/PlanUtil.scala     |   81 -
 .../calcite/FlinkCalciteCatalogReaderTest.java     |   84 -
 .../table/codegen/LongHashJoinGeneratorTest.java   |   92 -
 .../flink/table/codegen/SortCodeGeneratorTest.java |  583 ----
 .../flink/table/codegen/agg/TestLongAvgFunc.java   |   74 -
 .../aggfunctions/AggFunctionTestBase.java          |  284 --
 .../ConcatWithRetractAggFunctionTest.java          |   80 -
 .../ConcatWsWithRetractAggFunctionTest.java        |  159 -
 ...FirstLastValueAggFunctionWithOrderTestBase.java |  147 -
 .../FirstValueAggFunctionWithOrderTest.java        |  466 ---
 .../FirstValueAggFunctionWithoutOrderTest.java     |  351 ---
 ...stValueWithRetractAggFunctionWithOrderTest.java |  474 ---
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 ---
 .../LastValueAggFunctionWithOrderTest.java         |  467 ---
 .../LastValueAggFunctionWithoutOrderTest.java      |  352 ---
 ...stValueWithRetractAggFunctionWithOrderTest.java |  480 ---
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 ---
 .../MaxWithRetractAggFunctionTest.java             |  564 ----
 .../MinWithRetractAggFunctionTest.java             |  556 ----
 .../plan/nodes/resource/MockNodeTestBase.java      |  185 --
 .../parallelism/FinalParallelismSetterTest.java    |  118 -
 .../parallelism/ShuffleStageGeneratorTest.java     |  323 --
 .../ShuffleStageParallelismCalculatorTest.java     |  123 -
 .../plan/util/JavaUserDefinedAggFunctions.java     |  421 ---
 .../planner/codegen/LongHashJoinGeneratorTest.java |   92 +
 .../planner/codegen/SortCodeGeneratorTest.java     |  583 ++++
 .../table/planner/codegen/agg/TestLongAvgFunc.java |   74 +
 .../aggfunctions/AggFunctionTestBase.java          |  284 ++
 .../ConcatWithRetractAggFunctionTest.java          |   80 +
 .../ConcatWsWithRetractAggFunctionTest.java        |  159 +
 ...FirstLastValueAggFunctionWithOrderTestBase.java |  147 +
 .../FirstValueAggFunctionWithOrderTest.java        |  466 +++
 .../FirstValueAggFunctionWithoutOrderTest.java     |  351 +++
 ...stValueWithRetractAggFunctionWithOrderTest.java |  474 +++
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 +++
 .../LastValueAggFunctionWithOrderTest.java         |  467 +++
 .../LastValueAggFunctionWithoutOrderTest.java      |  352 +++
 ...stValueWithRetractAggFunctionWithOrderTest.java |  480 +++
 ...alueWithRetractAggFunctionWithoutOrderTest.java |  357 +++
 .../MaxWithRetractAggFunctionTest.java             |  564 ++++
 .../MinWithRetractAggFunctionTest.java             |  556 ++++
 .../plan/FlinkCalciteCatalogReaderTest.java        |   86 +
 .../plan/nodes/resource/MockNodeTestBase.java      |  185 ++
 .../parallelism/FinalParallelismSetterTest.java    |  118 +
 .../parallelism/ShuffleStageGeneratorTest.java     |  323 ++
 .../ShuffleStageParallelismCalculatorTest.java     |  123 +
 .../plan/utils/JavaUserDefinedAggFunctions.java    |  421 +++
 .../runtime/utils/BatchAbstractTestBase.java       |   54 +
 .../runtime/utils/FailingCollectionSource.java     |  251 ++
 .../table/planner/runtime/utils/JavaPojos.java     |   45 +
 .../runtime/utils/JavaUserDefinedAggFunctions.java |  428 +++
 .../utils/JavaUserDefinedScalarFunctions.java      |  157 +
 .../utils/JavaUserDefinedTableFunctions.java       |  142 +
 .../planner/runtime/utils/RangeInputFormat.java    |   67 +
 .../flink/table/planner/utils/BaseRowTestUtil.java |  108 +
 .../flink/table/planner/utils/DiffRepository.java  |  784 +++++
 .../table/runtime/utils/BatchAbstractTestBase.java |   54 -
 .../runtime/utils/FailingCollectionSource.java     |  251 --
 .../flink/table/runtime/utils/JavaPojos.java       |   45 -
 .../runtime/utils/JavaUserDefinedAggFunctions.java |  428 ---
 .../utils/JavaUserDefinedScalarFunctions.java      |  157 -
 .../utils/JavaUserDefinedTableFunctions.java       |  142 -
 .../table/runtime/utils/RangeInputFormat.java      |   67 -
 .../apache/flink/table/util/BaseRowTestUtil.java   |  108 -
 .../apache/flink/table/util/DiffRepository.java    |  784 -----
 .../org.apache.flink.table.factories.TableFactory  |    2 +-
 .../table/plan/batch/sql/SetOperatorsTest.xml      |  282 --
 .../flink/table/plan/batch/table/CalcTest.xml      |  235 --
 .../flink/table/plan/batch/table/CorrelateTest.xml |  130 -
 .../flink/table/plan/batch/table/JoinTest.xml      |  280 --
 .../table/plan/batch/table/SetOperatorsTest.xml    |  201 --
 .../stringexpr/CorrelateStringExpressionTest.xml   |  156 -
 .../PushFilterIntoTableSourceScanRuleTest.xml      |  172 --
 .../rules/logical/RewriteIntersectAllRuleTest.xml  |  151 -
 .../plan/rules/logical/RewriteMinusAllRuleTest.xml |  151 -
 .../table/plan/stream/sql/SetOperatorsTest.xml     |  282 --
 .../flink/table/plan/stream/table/CalcTest.xml     |  187 --
 .../plan/stream/table/ColumnFunctionsTest.xml      |  229 --
 .../table/plan/stream/table/CorrelateTest.xml      |  209 --
 .../table/plan/stream/table/OverWindowTest.xml     |  308 --
 .../stream => planner/plan/batch}/sql/CalcTest.xml |    0
 .../plan/batch/sql/DagOptimizationTest.xml         |    0
 .../plan/batch/sql/DeadlockBreakupTest.xml         |    0
 .../{ => planner}/plan/batch/sql/LimitTest.xml     |    0
 .../{ => planner}/plan/batch/sql/RankTest.xml      |    0
 .../plan/batch/sql/RemoveCollationTest.xml         |    0
 .../plan/batch/sql/RemoveShuffleTest.xml           |    0
 .../planner/plan/batch/sql/SetOperatorsTest.xml    |  282 ++
 .../{ => planner}/plan/batch/sql/SinkTest.xml      |    0
 .../{ => planner}/plan/batch/sql/SortLimitTest.xml |    0
 .../{ => planner}/plan/batch/sql/SortTest.xml      |    0
 .../plan/batch/sql/SubplanReuseTest.xml            |    0
 .../{ => planner}/plan/batch/sql/TableScanTest.xml |    0
 .../plan/batch/sql/TableSourceTest.xml             |    0
 .../plan/batch}/sql/UnionTest.xml                  |    0
 .../{ => planner}/plan/batch/sql/UnnestTest.xml    |    0
 .../{ => planner}/plan/batch/sql/ValuesTest.xml    |    0
 .../batch/sql/agg/AggregateReduceGroupingTest.xml  |    0
 .../plan/batch/sql/agg/DistinctAggregateTest.xml   |    0
 .../plan/batch/sql/agg/GroupingSetsTest.xml        |    0
 .../plan/batch/sql/agg/HashAggregateTest.xml       |    0
 .../plan/batch/sql/agg/OverAggregateTest.xml       |    0
 .../plan/batch/sql/agg/SortAggregateTest.xml       |    0
 .../plan/batch/sql/agg/WindowAggregateTest.xml     |    0
 .../plan/batch/sql/join/BroadcastHashJoinTest.xml  |    0
 .../sql/join/BroadcastHashSemiAntiJoinTest.xml     |    0
 .../plan/batch/sql/join/JoinReorderTest.xml        |    0
 .../plan/batch/sql/join/LookupJoinTest.xml         |    0
 .../plan/batch/sql/join/NestedLoopJoinTest.xml     |    0
 .../batch/sql/join/NestedLoopSemiAntiJoinTest.xml  |    0
 .../plan/batch/sql/join/SemiAntiJoinTest.xml       |    0
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   |    0
 .../sql/join/ShuffledHashSemiAntiJoinTest.xml      |    0
 .../plan/batch/sql/join/SingleRowJoinTest.xml      |    0
 .../plan/batch/sql/join/SortMergeJoinTest.xml      |    0
 .../batch/sql/join/SortMergeSemiAntiJoinTest.xml   |    0
 .../plan/batch/table/AggregateTest.xml             |    0
 .../table/planner/plan/batch/table/CalcTest.xml    |  235 ++
 .../plan/batch/table/ColumnFunctionsTest.xml       |    0
 .../planner/plan/batch/table/CorrelateTest.xml     |  130 +
 .../plan/batch/table/GroupWindowTest.xml           |    0
 .../table/planner/plan/batch/table/JoinTest.xml    |  280 ++
 .../planner/plan/batch/table/SetOperatorsTest.xml  |  201 ++
 .../stringexpr/CorrelateStringExpressionTest.xml   |  156 +
 .../batch/table/stringexpr/SetOperatorsTest.xml    |    0
 .../plan/nodes/resource/ExecNodeResourceTest.xml   |    0
 .../logical/AggregateReduceGroupingRuleTest.xml    |    0
 .../logical/CalcPruneAggregateCallRuleTest.xml     |    0
 .../rules/logical/CalcRankTransposeRuleTest.xml    |    0
 .../rules/logical/ConvertToNotInOrInRuleTest.xml   |    0
 .../logical/DecomposeGroupingSetsRuleTest.xml      |    0
 .../rules/logical/ExpressionReductionRulesTest.xml |    0
 ...nkAggregateExpandDistinctAggregatesRuleTest.xml |    0
 .../FlinkAggregateJoinTransposeRuleTest.xml        |    0
 .../rules/logical/FlinkAggregateRemoveRuleTest.xml |    0
 .../plan/rules/logical/FlinkCalcMergeRuleTest.xml  |    0
 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml |    0
 .../logical/FlinkJoinPushExpressionsRuleTest.xml   |    0
 .../rules/logical/FlinkLimit0RemoveRuleTest.xml    |    0
 .../FlinkLogicalRankRuleForConstantRangeTest.xml   |    0
 .../FlinkLogicalRankRuleForRangeEndTest.xml        |    0
 .../rules/logical/FlinkPruneEmptyRulesTest.xml     |    0
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.xml   |    0
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.xml     |    0
 .../FlinkSemiAntiJoinProjectTransposeRuleTest.xml  |    0
 .../JoinConditionEqualityTransferRuleTest.xml      |    0
 .../logical/JoinConditionTypeCoerceRuleTest.xml    |    0
 .../JoinDependentConditionDerivationRuleTest.xml   |    0
 .../rules/logical/JoinDeriveNullFilterRuleTest.xml |    0
 .../plan/rules/logical/LogicalUnnestRuleTest.xml   |    0
 .../logical/ProjectPruneAggregateCallRuleTest.xml  |    0
 .../ProjectSemiAntiJoinTransposeRuleTest.xml       |    0
 .../PushFilterIntoTableSourceScanRuleTest.xml      |  172 ++
 .../PushPartitionIntoTableSourceScanRuleTest.xml   |    0
 .../PushProjectIntoTableSourceScanRuleTest.xml     |    0
 .../logical/RankNumberColumnRemoveRuleTest.xml     |    0
 .../ReplaceIntersectWithSemiJoinRuleTest.xml       |    0
 .../logical/ReplaceMinusWithAntiJoinRuleTest.xml   |    0
 .../plan/rules/logical/RewriteCoalesceRuleTest.xml |    0
 .../rules/logical/RewriteIntersectAllRuleTest.xml  |  151 +
 .../plan/rules/logical/RewriteMinusAllRuleTest.xml |  151 +
 .../logical/RewriteMultiJoinConditionRuleTest.xml  |    0
 .../logical/SimplifyFilterConditionRuleTest.xml    |    0
 .../logical/SimplifyJoinConditionRuleTest.xml      |    0
 .../plan/rules/logical/SplitAggregateRuleTest.xml  |    0
 .../rules/logical/WindowGroupReorderRuleTest.xml   |    0
 .../subquery/FlinkRewriteSubQueryRuleTest.xml      |    0
 .../logical/subquery/SubQueryAntiJoinTest.xml      |    0
 .../logical/subquery/SubQuerySemiJoinTest.xml      |    0
 .../SubqueryCorrelateVariablesValidationTest.xml   |    0
 .../batch/RemoveRedundantLocalHashAggRuleTest.xml  |    0
 .../batch/RemoveRedundantLocalRankRuleTest.xml     |    0
 .../batch/RemoveRedundantLocalSortAggRuleTest.xml  |    0
 .../rules/physical/stream/RetractionRulesTest.xml  |    0
 .../stream/RetractionRulesWithTwoStageAggTest.xml  |    0
 .../batch => planner/plan/stream}/sql/CalcTest.xml |    0
 .../plan/stream/sql/DagOptimizationTest.xml        |    0
 .../{ => planner}/plan/stream/sql/LimitTest.xml    |    0
 .../plan/stream/sql/MiniBatchIntervalInferTest.xml |    0
 .../plan/stream/sql/ModifiedMonotonicityTest.xml   |    0
 .../{ => planner}/plan/stream/sql/RankTest.xml     |    0
 .../stream/sql/RelTimeIndicatorConverterTest.xml   |    0
 .../planner/plan/stream/sql/SetOperatorsTest.xml   |  282 ++
 .../{ => planner}/plan/stream/sql/SinkTest.xml     |    0
 .../plan/stream/sql/SortLimitTest.xml              |    0
 .../{ => planner}/plan/stream/sql/SortTest.xml     |    0
 .../plan/stream/sql/SubplanReuseTest.xml           |    0
 .../plan/stream/sql/TableScanTest.xml              |    0
 .../plan/stream/sql/TableSourceTest.xml            |    0
 .../plan/stream}/sql/UnionTest.xml                 |    0
 .../{ => planner}/plan/stream/sql/UnnestTest.xml   |    0
 .../{ => planner}/plan/stream/sql/ValuesTest.xml   |    0
 .../plan/stream/sql/agg/AggregateTest.xml          |    0
 .../plan/stream/sql/agg/DistinctAggregateTest.xml  |    0
 .../plan/stream/sql/agg/GroupingSetsTest.xml       |    0
 .../stream/sql/agg/IncrementalAggregateTest.xml    |    0
 .../plan/stream/sql/agg/OverAggregateTest.xml      |    0
 .../plan/stream/sql/agg/TwoStageAggregateTest.xml  |    0
 .../plan/stream/sql/agg/WindowAggregateTest.xml    |    0
 .../plan/stream/sql/join/JoinReorderTest.xml       |    0
 .../plan/stream/sql/join/JoinTest.xml              |    0
 .../plan/stream/sql/join/LookupJoinTest.xml        |    0
 .../plan/stream/sql/join/SemiAntiJoinTest.xml      |    0
 .../plan/stream/sql/join/TemporalJoinTest.xml      |    0
 .../plan/stream/sql/join/WindowJoinTest.xml        |    0
 .../plan/stream/table/AggregateTest.xml            |    0
 .../table/planner/plan/stream/table/CalcTest.xml   |  187 ++
 .../plan/stream/table/ColumnFunctionsTest.xml      |  229 ++
 .../planner/plan/stream/table/CorrelateTest.xml    |  209 ++
 .../plan/stream/table/GroupWindowTest.xml          |    0
 .../{ => planner}/plan/stream/table/JoinTest.xml   |    0
 .../planner/plan/stream/table/OverWindowTest.xml   |  308 ++
 .../plan/stream/table/SetOperatorsTest.xml         |    0
 .../plan/stream/table/TableSourceTest.xml          |    0
 .../plan/stream/table/TwoStageAggregateTest.xml    |    0
 .../flink/table/api/TableEnvironmentTest.scala     |    2 +-
 .../apache/flink/table/api/batch/ExplainTest.scala |    2 +-
 .../flink/table/api/stream/ExplainTest.scala       |    2 +-
 .../validation/MatchRecognizeValidationTest.scala  |    6 +-
 .../sql/validation/OverWindowValidationTest.scala  |    5 +-
 .../table/calcite/CalciteConfigBuilderTest.scala   |  248 --
 .../flink/table/calcite/FlinkTypeFactoryTest.scala |   85 -
 .../flink/table/catalog/CatalogTableITCase.scala   |  535 ----
 .../table/codegen/HashCodeGeneratorTest.scala      |   62 -
 .../codegen/ProjectionCodeGeneratorTest.scala      |  110 -
 .../flink/table/codegen/agg/AggTestBase.scala      |  112 -
 .../codegen/agg/AggsHandlerCodeGeneratorTest.scala |  102 -
 .../codegen/agg/batch/AggWithoutKeysTest.scala     |  104 -
 .../table/codegen/agg/batch/BatchAggTestBase.scala |   99 -
 .../agg/batch/HashAggCodeGeneratorTest.scala       |  131 -
 .../agg/batch/SortAggCodeGeneratorTest.scala       |  120 -
 .../flink/table/expressions/ArrayTypeTest.scala    |  215 --
 .../table/expressions/CompositeAccessTest.scala    |  141 -
 .../flink/table/expressions/DecimalTypeTest.scala  |  223 --
 .../flink/table/expressions/KeywordParseTest.scala |   62 -
 .../flink/table/expressions/LiteralTest.scala      |  157 -
 .../flink/table/expressions/MapTypeTest.scala      |  193 --
 .../table/expressions/MathFunctionsTest.scala      |  693 -----
 .../table/expressions/NonDeterministicTests.scala  |   79 -
 .../flink/table/expressions/RowTypeTest.scala      |   88 -
 .../table/expressions/ScalarFunctionsTest.scala    | 3181 --------------------
 .../table/expressions/ScalarOperatorsTest.scala    |  125 -
 .../table/expressions/SqlExpressionTest.scala      |  314 --
 .../table/expressions/TemporalTypesTest.scala      |  756 -----
 .../expressions/utils/ArrayTypeTestBase.scala      |   63 -
 .../expressions/utils/CompositeTypeTestBase.scala  |   92 -
 .../expressions/utils/ExpressionTestBase.scala     |  211 --
 .../table/expressions/utils/MapTypeTestBase.scala  |   75 -
 .../table/expressions/utils/RowTypeTestBase.scala  |   67 -
 .../utils/ScalarOperatorsTestBase.scala            |   75 -
 .../expressions/utils/ScalarTypesTestBase.scala    |  149 -
 .../utils/userDefinedScalarFunctions.scala         |  412 ---
 .../validation/ArrayTypeValidationTest.scala       |   58 -
 .../validation/CompositeAccessValidationTest.scala |   38 -
 .../validation/MapTypeValidationTest.scala         |   46 -
 .../validation/RowTypeValidationTest.scala         |   41 -
 .../validation/ScalarFunctionsValidationTest.scala |  149 -
 .../utils/TestCollectionTableFactory.scala         |  269 --
 .../flink/table/match/PatternTranslatorTest.scala  |  397 ---
 .../table/match/PatternTranslatorTestBase.scala    |  140 -
 .../flink/table/plan/batch/sql/CalcTest.scala      |  161 -
 .../table/plan/batch/sql/DagOptimizationTest.scala |  471 ---
 .../table/plan/batch/sql/DeadlockBreakupTest.scala |  184 --
 .../flink/table/plan/batch/sql/LimitTest.scala     |   93 -
 .../flink/table/plan/batch/sql/RankTest.scala      |  171 --
 .../table/plan/batch/sql/RemoveCollationTest.scala |  387 ---
 .../table/plan/batch/sql/RemoveShuffleTest.scala   |  549 ----
 .../table/plan/batch/sql/SetOperatorsTest.scala    |  131 -
 .../flink/table/plan/batch/sql/SinkTest.scala      |   91 -
 .../flink/table/plan/batch/sql/SortLimitTest.scala |   95 -
 .../flink/table/plan/batch/sql/SortTest.scala      |   75 -
 .../table/plan/batch/sql/SubplanReuseTest.scala    |  446 ---
 .../flink/table/plan/batch/sql/TableScanTest.scala |   37 -
 .../table/plan/batch/sql/TableSourceTest.scala     |  223 --
 .../flink/table/plan/batch/sql/UnionTest.scala     |   66 -
 .../flink/table/plan/batch/sql/UnnestTest.scala    |   26 -
 .../flink/table/plan/batch/sql/ValuesTest.scala    |   49 -
 .../sql/agg/AggregateReduceGroupingTest.scala      |   24 -
 .../plan/batch/sql/agg/AggregateTestBase.scala     |  203 --
 .../plan/batch/sql/agg/DistinctAggregateTest.scala |   85 -
 .../plan/batch/sql/agg/GroupingSetsTest.scala      |  458 ---
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   74 -
 .../plan/batch/sql/agg/OverAggregateTest.scala     |  344 ---
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   55 -
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |  315 --
 .../batch/sql/join/BroadcastHashJoinTest.scala     |  141 -
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |  180 --
 .../plan/batch/sql/join/JoinReorderTest.scala      |   25 -
 .../table/plan/batch/sql/join/JoinTestBase.scala   |  212 --
 .../table/plan/batch/sql/join/LookupJoinTest.scala |  282 --
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   32 -
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   32 -
 .../plan/batch/sql/join/SemiAntiJoinTest.scala     |   26 -
 .../plan/batch/sql/join/SemiAntiJoinTestBase.scala |  585 ----
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |  119 -
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |  186 --
 .../plan/batch/sql/join/SingleRowJoinTest.scala    |   96 -
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |  111 -
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |  163 -
 .../plan/batch/sql/join/TemporalJoinTest.scala     |  112 -
 .../table/plan/batch/table/AggregateTest.scala     |   75 -
 .../flink/table/plan/batch/table/CalcTest.scala    |  203 --
 .../plan/batch/table/ColumnFunctionsTest.scala     |   52 -
 .../table/plan/batch/table/CorrelateTest.scala     |  120 -
 .../table/plan/batch/table/GroupWindowTest.scala   |  158 -
 .../flink/table/plan/batch/table/JoinTest.scala    |  212 --
 .../table/plan/batch/table/SetOperatorsTest.scala  |  133 -
 .../plan/batch/table/TemporalTableJoinTest.scala   |   72 -
 .../stringexpr/AggregateStringExpressionTest.scala |  341 ---
 .../stringexpr/CalcStringExpressionTest.scala      |  366 ---
 .../stringexpr/CorrelateStringExpressionTest.scala |   94 -
 .../stringexpr/JoinStringExpressionTest.scala      |  187 --
 .../batch/table/stringexpr/SetOperatorsTest.scala  |   52 -
 .../stringexpr/SortStringExpressionTest.scala      |   61 -
 .../table/validation/AggregateValidationTest.scala |  221 --
 .../table/validation/CalcValidationTest.scala      |  117 -
 .../table/validation/CorrelateValidationTest.scala |   46 -
 .../validation/GroupWindowValidationTest.scala     |  172 --
 .../table/validation/JoinValidationTest.scala      |  118 -
 .../validation/OverWindowValidationTest.scala      |   56 -
 .../validation/SetOperatorsValidationTest.scala    |  113 -
 .../table/validation/SortValidationTest.scala      |   69 -
 .../common/AggregateReduceGroupingTestBase.scala   |  314 --
 .../table/plan/common/JoinReorderTestBase.scala    |  234 --
 .../flink/table/plan/common/UnnestTestBase.scala   |  129 -
 .../flink/table/plan/cost/FlinkCostTest.scala      |  187 --
 .../metadata/AggCallSelectivityEstimatorTest.scala |  630 ----
 .../metadata/FlinkRelMdColumnIntervalTest.scala    |  608 ----
 .../metadata/FlinkRelMdColumnNullCountTest.scala   |  287 --
 .../FlinkRelMdColumnOriginNullCountTest.scala      |  144 -
 .../metadata/FlinkRelMdColumnUniquenessTest.scala  |  611 ----
 .../metadata/FlinkRelMdCumulativeCostTest.scala    |   52 -
 .../metadata/FlinkRelMdDistinctRowCountTest.scala  |  640 ----
 .../plan/metadata/FlinkRelMdDistributionTest.scala |  109 -
 .../FlinkRelMdFilteredColumnIntervalTest.scala     |  187 --
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  | 2305 --------------
 .../FlinkRelMdModifiedMonotonicityTest.scala       |  290 --
 .../metadata/FlinkRelMdNonCumulativeCostTest.scala |   51 -
 .../FlinkRelMdPercentageOriginalRowsTest.scala     |   88 -
 .../metadata/FlinkRelMdPopulationSizeTest.scala    |  355 ---
 .../plan/metadata/FlinkRelMdRowCountTest.scala     |  253 --
 .../plan/metadata/FlinkRelMdSelectivityTest.scala  |  535 ----
 .../table/plan/metadata/FlinkRelMdSizeTest.scala   |  209 --
 .../plan/metadata/FlinkRelMdUniqueGroupsTest.scala |  646 ----
 .../plan/metadata/FlinkRelMdUniqueKeysTest.scala   |  273 --
 .../metadata/MetadataHandlerConsistencyTest.scala  |  150 -
 .../table/plan/metadata/MetadataTestUtil.scala     |  269 --
 .../plan/metadata/SelectivityEstimatorTest.scala   | 1068 -------
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  250 --
 .../optimize/program/FlinkChainedProgramTest.scala |  157 -
 .../program/FlinkHepRuleSetProgramTest.scala       |  103 -
 .../optimize/program/FlinkVolcanoProgramTest.scala |   51 -
 .../logical/AggregateReduceGroupingRuleTest.scala  |   44 -
 .../logical/CalcPruneAggregateCallRuleTest.scala   |   50 -
 .../rules/logical/CalcRankTransposeRuleTest.scala  |  191 --
 .../rules/logical/ConvertToNotInOrInRuleTest.scala |  168 --
 .../logical/DecomposeGroupingSetsRuleTest.scala    |  145 -
 .../logical/ExpressionReductionRulesTest.scala     |   49 -
 ...AggregateExpandDistinctAggregatesRuleTest.scala |  174 --
 .../FlinkAggregateJoinTransposeRuleTest.scala      |  147 -
 .../logical/FlinkAggregateRemoveRuleTest.scala     |  235 --
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   85 -
 .../rules/logical/FlinkFilterJoinRuleTest.scala    |  158 -
 .../logical/FlinkJoinPushExpressionsRuleTest.scala |   80 -
 .../rules/logical/FlinkLimit0RemoveRuleTest.scala  |   99 -
 .../FlinkLogicalRankRuleForConstantRangeTest.scala |  211 --
 .../FlinkLogicalRankRuleForRangeEndTest.scala      |  196 --
 .../rules/logical/FlinkPruneEmptyRulesTest.scala   |   71 -
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.scala |   76 -
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.scala   |  359 ---
 ...FlinkSemiAntiJoinProjectTransposeRuleTest.scala |   81 -
 .../JoinConditionEqualityTransferRuleTest.scala    |  146 -
 .../logical/JoinConditionTypeCoerceRuleTest.scala  |  116 -
 .../JoinDependentConditionDerivationRuleTest.scala |  120 -
 .../logical/JoinDeriveNullFilterRuleTest.scala     |  118 -
 .../plan/rules/logical/LogicalUnnestRuleTest.scala |   50 -
 .../ProjectPruneAggregateCallRuleTest.scala        |   50 -
 .../ProjectSemiAntiJoinTransposeRuleTest.scala     |  151 -
 .../logical/PruneAggregateCallRuleTestBase.scala   |  176 --
 .../PushFilterIntoTableSourceScanRuleTest.scala    |   97 -
 .../PushPartitionIntoTableSourceScanRuleTest.scala |   96 -
 .../PushProjectIntoTableSourceScanRuleTest.scala   |  127 -
 .../logical/RankNumberColumnRemoveRuleTest.scala   |   97 -
 .../ReplaceIntersectWithSemiJoinRuleTest.scala     |   74 -
 .../logical/ReplaceMinusWithAntiJoinRuleTest.scala |   74 -
 .../rules/logical/RewriteCoalesceRuleTest.scala    |  141 -
 .../logical/RewriteIntersectAllRuleTest.scala      |   75 -
 .../rules/logical/RewriteMinusAllRuleTest.scala    |   73 -
 .../RewriteMultiJoinConditionRuleTest.scala        |  151 -
 .../logical/SimplifyFilterConditionRuleTest.scala  |  108 -
 .../logical/SimplifyJoinConditionRuleTest.scala    |   68 -
 .../rules/logical/SplitAggregateRuleTest.scala     |  186 --
 .../rules/logical/WindowGroupReorderRuleTest.scala |  180 --
 .../subquery/FlinkRewriteSubQueryRuleTest.scala    |  212 --
 .../logical/subquery/SubQueryAntiJoinTest.scala    |  770 -----
 .../logical/subquery/SubQuerySemiJoinTest.scala    | 1674 ----------
 .../rules/logical/subquery/SubQueryTestBase.scala  |   44 -
 .../SubqueryCorrelateVariablesValidationTest.scala |  130 -
 .../RemoveRedundantLocalHashAggRuleTest.scala      |   72 -
 .../batch/RemoveRedundantLocalRankRuleTest.scala   |   73 -
 .../RemoveRedundantLocalSortAggRuleTest.scala      |   67 -
 .../physical/stream/RetractionRulesTest.scala      |   70 -
 .../RetractionRulesWithTwoStageAggTest.scala       |   76 -
 .../plan/schema/TimeIndicatorRelDataTypeTest.scala |   42 -
 .../flink/table/plan/stats/ValueIntervalTest.scala |  459 ---
 .../flink/table/plan/stream/sql/CalcTest.scala     |  159 -
 .../plan/stream/sql/DagOptimizationTest.scala      |  530 ----
 .../flink/table/plan/stream/sql/LimitTest.scala    |  213 --
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  352 ---
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  276 --
 .../flink/table/plan/stream/sql/RankTest.scala     |  718 -----
 .../stream/sql/RelTimeIndicatorConverterTest.scala |  178 --
 .../table/plan/stream/sql/SetOperatorsTest.scala   |  128 -
 .../flink/table/plan/stream/sql/SinkTest.scala     |  153 -
 .../table/plan/stream/sql/SortLimitTest.scala      |  327 --
 .../flink/table/plan/stream/sql/SortTest.scala     |   79 -
 .../table/plan/stream/sql/SubplanReuseTest.scala   |  300 --
 .../table/plan/stream/sql/TableScanTest.scala      |   42 -
 .../table/plan/stream/sql/TableSourceTest.scala    |  395 ---
 .../flink/table/plan/stream/sql/UnionTest.scala    |   66 -
 .../flink/table/plan/stream/sql/UnnestTest.scala   |   26 -
 .../flink/table/plan/stream/sql/ValuesTest.scala   |   49 -
 .../table/plan/stream/sql/agg/AggregateTest.scala  |  263 --
 .../stream/sql/agg/DistinctAggregateTest.scala     |  224 --
 .../plan/stream/sql/agg/GroupingSetsTest.scala     |  458 ---
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   52 -
 .../plan/stream/sql/agg/OverAggregateTest.scala    |  433 ---
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   79 -
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  298 --
 .../plan/stream/sql/join/JoinReorderTest.scala     |   25 -
 .../table/plan/stream/sql/join/JoinTest.scala      |  284 --
 .../plan/stream/sql/join/LookupJoinTest.scala      |  493 ---
 .../plan/stream/sql/join/SemiAntiJoinTest.scala    |  576 ----
 .../plan/stream/sql/join/TemporalJoinTest.scala    |  132 -
 .../plan/stream/sql/join/WindowJoinTest.scala      |  453 ---
 .../table/plan/stream/table/AggregateTest.scala    |  230 --
 .../flink/table/plan/stream/table/CalcTest.scala   |  161 -
 .../plan/stream/table/ColumnFunctionsTest.scala    |  229 --
 .../table/plan/stream/table/CorrelateTest.scala    |  181 --
 .../table/plan/stream/table/GroupWindowTest.scala  |  409 ---
 .../flink/table/plan/stream/table/JoinTest.scala   |  263 --
 .../table/plan/stream/table/OverWindowTest.scala   |  222 --
 .../table/plan/stream/table/SetOperatorsTest.scala |   87 -
 .../table/plan/stream/table/TableSourceTest.scala  |  302 --
 .../plan/stream/table/TemporalTableJoinTest.scala  |  191 --
 .../plan/stream/table/TwoStageAggregateTest.scala  |  112 -
 .../stringexpr/AggregateStringExpressionTest.scala |  246 --
 .../stringexpr/CalcStringExpressionTest.scala      |  183 --
 .../stringexpr/CorrelateStringExpressionTest.scala |  160 -
 .../GroupWindowStringExpressionTest.scala          |  263 --
 ...pWindowTableAggregateStringExpressionTest.scala |  227 --
 .../OverWindowStringExpressionTest.scala           |  248 --
 .../SetOperatorsStringExpressionTest.scala         |   50 -
 .../table/validation/AggregateValidationTest.scala |  127 -
 .../table/validation/CalcValidationTest.scala      |  165 -
 .../table/validation/CorrelateValidationTest.scala |  178 --
 .../validation/GroupWindowValidationTest.scala     |  308 --
 .../validation/OverWindowValidationTest.scala      |  164 -
 .../validation/SetOperatorsValidationTest.scala    |   80 -
 .../table/validation/TableSinkValidationTest.scala |   87 -
 .../TemporalTableJoinValidationTest.scala          |  114 -
 .../validation/UnsupportedOpsValidationTest.scala  |  109 -
 .../plan/trait/FlinkRelDistributionTest.scala      |  165 -
 .../table/plan/util/FlinkRelOptUtilTest.scala      |  141 -
 .../flink/table/plan/util/FlinkRexUtilTest.scala   |  407 ---
 .../flink/table/plan/util/InputTypeBuilder.scala   |   53 -
 .../table/plan/util/PartitionPrunerTest.scala      |  111 -
 .../flink/table/plan/util/RelDigestUtilTest.scala  |  108 -
 .../table/plan/util/RexNodeExtractorTest.scala     |  903 ------
 .../table/plan/util/RexNodeRewriterTest.scala      |   75 -
 .../flink/table/plan/util/RexNodeTestBase.scala    |   90 -
 .../org/apache/flink/table/plan/util/pojos.scala   |   51 -
 .../planner/calcite/CalciteConfigBuilderTest.scala |  249 ++
 .../planner/calcite/FlinkTypeFactoryTest.scala     |   86 +
 .../table/planner/catalog/CatalogTableITCase.scala |  535 ++++
 .../planner/codegen/HashCodeGeneratorTest.scala    |   62 +
 .../codegen/ProjectionCodeGeneratorTest.scala      |  110 +
 .../table/planner/codegen/agg/AggTestBase.scala    |  112 +
 .../codegen/agg/AggsHandlerCodeGeneratorTest.scala |  102 +
 .../codegen/agg/batch/AggWithoutKeysTest.scala     |  104 +
 .../codegen/agg/batch/BatchAggTestBase.scala       |   99 +
 .../agg/batch/HashAggCodeGeneratorTest.scala       |  131 +
 .../agg/batch/SortAggCodeGeneratorTest.scala       |  120 +
 .../table/planner/expressions/ArrayTypeTest.scala  |  216 ++
 .../planner/expressions/CompositeAccessTest.scala  |  142 +
 .../planner/expressions/DecimalTypeTest.scala      |  223 ++
 .../planner/expressions/KeywordParseTest.scala     |   63 +
 .../table/planner/expressions/LiteralTest.scala    |  158 +
 .../table/planner/expressions/MapTypeTest.scala    |  194 ++
 .../planner/expressions/MathFunctionsTest.scala    |  694 +++++
 .../expressions/NonDeterministicTests.scala        |   80 +
 .../table/planner/expressions/RowTypeTest.scala    |   89 +
 .../planner/expressions/ScalarFunctionsTest.scala  | 3181 ++++++++++++++++++++
 .../planner/expressions/ScalarOperatorsTest.scala  |  126 +
 .../planner/expressions/SqlExpressionTest.scala    |  314 ++
 .../planner/expressions/TemporalTypesTest.scala    |  756 +++++
 .../expressions/utils/ArrayTypeTestBase.scala      |   63 +
 .../expressions/utils/CompositeTypeTestBase.scala  |   92 +
 .../expressions/utils/ExpressionTestBase.scala     |  211 ++
 .../expressions/utils/MapTypeTestBase.scala        |   76 +
 .../expressions/utils/RowTypeTestBase.scala        |   67 +
 .../utils/ScalarOperatorsTestBase.scala            |   75 +
 .../expressions/utils/ScalarTypesTestBase.scala    |  149 +
 .../utils/userDefinedScalarFunctions.scala         |  412 +++
 .../validation/ArrayTypeValidationTest.scala       |   59 +
 .../validation/CompositeAccessValidationTest.scala |   39 +
 .../validation/MapTypeValidationTest.scala         |   47 +
 .../validation/RowTypeValidationTest.scala         |   42 +
 .../validation/ScalarFunctionsValidationTest.scala |  150 +
 .../utils/TestCollectionTableFactory.scala         |  269 ++
 .../planner/match/PatternTranslatorTest.scala      |  398 +++
 .../planner/match/PatternTranslatorTestBase.scala  |  140 +
 .../table/planner/plan/batch/sql/CalcTest.scala    |  161 +
 .../plan/batch/sql/DagOptimizationTest.scala       |  471 +++
 .../plan/batch/sql/DeadlockBreakupTest.scala       |  184 ++
 .../table/planner/plan/batch/sql/LimitTest.scala   |   93 +
 .../table/planner/plan/batch/sql/RankTest.scala    |  171 ++
 .../plan/batch/sql/RemoveCollationTest.scala       |  388 +++
 .../planner/plan/batch/sql/RemoveShuffleTest.scala |  550 ++++
 .../planner/plan/batch/sql/SetOperatorsTest.scala  |  131 +
 .../table/planner/plan/batch/sql/SinkTest.scala    |   93 +
 .../planner/plan/batch/sql/SortLimitTest.scala     |   95 +
 .../table/planner/plan/batch/sql/SortTest.scala    |   75 +
 .../planner/plan/batch/sql/SubplanReuseTest.scala  |  446 +++
 .../planner/plan/batch/sql/TableScanTest.scala     |   37 +
 .../planner/plan/batch/sql/TableSourceTest.scala   |  223 ++
 .../table/planner/plan/batch/sql/UnionTest.scala   |   66 +
 .../table/planner/plan/batch/sql/UnnestTest.scala  |   26 +
 .../table/planner/plan/batch/sql/ValuesTest.scala  |   49 +
 .../sql/agg/AggregateReduceGroupingTest.scala      |   24 +
 .../plan/batch/sql/agg/AggregateTestBase.scala     |  203 ++
 .../plan/batch/sql/agg/DistinctAggregateTest.scala |   85 +
 .../plan/batch/sql/agg/GroupingSetsTest.scala      |  458 +++
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   74 +
 .../plan/batch/sql/agg/OverAggregateTest.scala     |  344 +++
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   55 +
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |  315 ++
 .../batch/sql/join/BroadcastHashJoinTest.scala     |  141 +
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |  180 ++
 .../plan/batch/sql/join/JoinReorderTest.scala      |   25 +
 .../planner/plan/batch/sql/join/JoinTestBase.scala |  212 ++
 .../plan/batch/sql/join/LookupJoinTest.scala       |  282 ++
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   32 +
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   32 +
 .../plan/batch/sql/join/SemiAntiJoinTest.scala     |   26 +
 .../plan/batch/sql/join/SemiAntiJoinTestBase.scala |  585 ++++
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |  119 +
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |  186 ++
 .../plan/batch/sql/join/SingleRowJoinTest.scala    |   96 +
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |  111 +
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |  163 +
 .../plan/batch/sql/join/TemporalJoinTest.scala     |  112 +
 .../planner/plan/batch/table/AggregateTest.scala   |   75 +
 .../table/planner/plan/batch/table/CalcTest.scala  |  203 ++
 .../plan/batch/table/ColumnFunctionsTest.scala     |   52 +
 .../planner/plan/batch/table/CorrelateTest.scala   |  120 +
 .../planner/plan/batch/table/GroupWindowTest.scala |  158 +
 .../table/planner/plan/batch/table/JoinTest.scala  |  212 ++
 .../plan/batch/table/SetOperatorsTest.scala        |  133 +
 .../plan/batch/table/TemporalTableJoinTest.scala   |   72 +
 .../stringexpr/AggregateStringExpressionTest.scala |  341 +++
 .../stringexpr/CalcStringExpressionTest.scala      |  366 +++
 .../stringexpr/CorrelateStringExpressionTest.scala |   94 +
 .../stringexpr/JoinStringExpressionTest.scala      |  187 ++
 .../batch/table/stringexpr/SetOperatorsTest.scala  |   52 +
 .../stringexpr/SortStringExpressionTest.scala      |   61 +
 .../table/validation/AggregateValidationTest.scala |  221 ++
 .../table/validation/CalcValidationTest.scala      |  117 +
 .../table/validation/CorrelateValidationTest.scala |   46 +
 .../validation/GroupWindowValidationTest.scala     |  172 ++
 .../table/validation/JoinValidationTest.scala      |  118 +
 .../validation/OverWindowValidationTest.scala      |   56 +
 .../validation/SetOperatorsValidationTest.scala    |  113 +
 .../table/validation/SortValidationTest.scala      |   69 +
 .../common/AggregateReduceGroupingTestBase.scala   |  315 ++
 .../planner/plan/common/JoinReorderTestBase.scala  |  235 ++
 .../table/planner/plan/common/UnnestTestBase.scala |  129 +
 .../table/planner/plan/cost/FlinkCostTest.scala    |  187 ++
 .../metadata/AggCallSelectivityEstimatorTest.scala |  633 ++++
 .../metadata/FlinkRelMdColumnIntervalTest.scala    |  608 ++++
 .../metadata/FlinkRelMdColumnNullCountTest.scala   |  287 ++
 .../FlinkRelMdColumnOriginNullCountTest.scala      |  144 +
 .../metadata/FlinkRelMdColumnUniquenessTest.scala  |  610 ++++
 .../metadata/FlinkRelMdCumulativeCostTest.scala    |   52 +
 .../metadata/FlinkRelMdDistinctRowCountTest.scala  |  640 ++++
 .../plan/metadata/FlinkRelMdDistributionTest.scala |  109 +
 .../FlinkRelMdFilteredColumnIntervalTest.scala     |  187 ++
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  | 2308 ++++++++++++++
 .../FlinkRelMdModifiedMonotonicityTest.scala       |  290 ++
 .../metadata/FlinkRelMdNonCumulativeCostTest.scala |   51 +
 .../FlinkRelMdPercentageOriginalRowsTest.scala     |   88 +
 .../metadata/FlinkRelMdPopulationSizeTest.scala    |  355 +++
 .../plan/metadata/FlinkRelMdRowCountTest.scala     |  251 ++
 .../plan/metadata/FlinkRelMdSelectivityTest.scala  |  532 ++++
 .../planner/plan/metadata/FlinkRelMdSizeTest.scala |  209 ++
 .../plan/metadata/FlinkRelMdUniqueGroupsTest.scala |  643 ++++
 .../plan/metadata/FlinkRelMdUniqueKeysTest.scala   |  273 ++
 .../metadata/MetadataHandlerConsistencyTest.scala  |  150 +
 .../planner/plan/metadata/MetadataTestUtil.scala   |  270 ++
 .../plan/metadata/SelectivityEstimatorTest.scala   | 1071 +++++++
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  251 ++
 .../optimize/program/FlinkChainedProgramTest.scala |  157 +
 .../program/FlinkHepRuleSetProgramTest.scala       |  104 +
 .../optimize/program/FlinkVolcanoProgramTest.scala |   51 +
 .../logical/AggregateReduceGroupingRuleTest.scala  |   44 +
 .../logical/CalcPruneAggregateCallRuleTest.scala   |   50 +
 .../rules/logical/CalcRankTransposeRuleTest.scala  |  191 ++
 .../rules/logical/ConvertToNotInOrInRuleTest.scala |  168 ++
 .../logical/DecomposeGroupingSetsRuleTest.scala    |  145 +
 .../logical/ExpressionReductionRulesTest.scala     |   49 +
 ...AggregateExpandDistinctAggregatesRuleTest.scala |  174 ++
 .../FlinkAggregateJoinTransposeRuleTest.scala      |  147 +
 .../logical/FlinkAggregateRemoveRuleTest.scala     |  235 ++
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   85 +
 .../rules/logical/FlinkFilterJoinRuleTest.scala    |  158 +
 .../logical/FlinkJoinPushExpressionsRuleTest.scala |   80 +
 .../rules/logical/FlinkLimit0RemoveRuleTest.scala  |   99 +
 .../FlinkLogicalRankRuleForConstantRangeTest.scala |  211 ++
 .../FlinkLogicalRankRuleForRangeEndTest.scala      |  196 ++
 .../rules/logical/FlinkPruneEmptyRulesTest.scala   |   71 +
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.scala |   76 +
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.scala   |  359 +++
 ...FlinkSemiAntiJoinProjectTransposeRuleTest.scala |   81 +
 .../JoinConditionEqualityTransferRuleTest.scala    |  146 +
 .../logical/JoinConditionTypeCoerceRuleTest.scala  |  116 +
 .../JoinDependentConditionDerivationRuleTest.scala |  120 +
 .../logical/JoinDeriveNullFilterRuleTest.scala     |  119 +
 .../plan/rules/logical/LogicalUnnestRuleTest.scala |   50 +
 .../ProjectPruneAggregateCallRuleTest.scala        |   50 +
 .../ProjectSemiAntiJoinTransposeRuleTest.scala     |  151 +
 .../logical/PruneAggregateCallRuleTestBase.scala   |  176 ++
 .../PushFilterIntoTableSourceScanRuleTest.scala    |   97 +
 .../PushPartitionIntoTableSourceScanRuleTest.scala |   96 +
 .../PushProjectIntoTableSourceScanRuleTest.scala   |  127 +
 .../logical/RankNumberColumnRemoveRuleTest.scala   |   97 +
 .../ReplaceIntersectWithSemiJoinRuleTest.scala     |   74 +
 .../logical/ReplaceMinusWithAntiJoinRuleTest.scala |   74 +
 .../rules/logical/RewriteCoalesceRuleTest.scala    |  141 +
 .../logical/RewriteIntersectAllRuleTest.scala      |   75 +
 .../rules/logical/RewriteMinusAllRuleTest.scala    |   73 +
 .../RewriteMultiJoinConditionRuleTest.scala        |  151 +
 .../logical/SimplifyFilterConditionRuleTest.scala  |  108 +
 .../logical/SimplifyJoinConditionRuleTest.scala    |   68 +
 .../rules/logical/SplitAggregateRuleTest.scala     |  186 ++
 .../rules/logical/WindowGroupReorderRuleTest.scala |  180 ++
 .../subquery/FlinkRewriteSubQueryRuleTest.scala    |  212 ++
 .../logical/subquery/SubQueryAntiJoinTest.scala    |  769 +++++
 .../logical/subquery/SubQuerySemiJoinTest.scala    | 1674 ++++++++++
 .../rules/logical/subquery/SubQueryTestBase.scala  |   44 +
 .../SubqueryCorrelateVariablesValidationTest.scala |  130 +
 .../RemoveRedundantLocalHashAggRuleTest.scala      |   72 +
 .../batch/RemoveRedundantLocalRankRuleTest.scala   |   73 +
 .../RemoveRedundantLocalSortAggRuleTest.scala      |   67 +
 .../physical/stream/RetractionRulesTest.scala      |   70 +
 .../RetractionRulesWithTwoStageAggTest.scala       |   76 +
 .../plan/schema/TimeIndicatorRelDataTypeTest.scala |   42 +
 .../planner/plan/stats/ValueIntervalTest.scala     |  459 +++
 .../table/planner/plan/stream/sql/CalcTest.scala   |  159 +
 .../plan/stream/sql/DagOptimizationTest.scala      |  530 ++++
 .../table/planner/plan/stream/sql/LimitTest.scala  |  213 ++
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  352 +++
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  276 ++
 .../table/planner/plan/stream/sql/RankTest.scala   |  718 +++++
 .../stream/sql/RelTimeIndicatorConverterTest.scala |  178 ++
 .../planner/plan/stream/sql/SetOperatorsTest.scala |  128 +
 .../table/planner/plan/stream/sql/SinkTest.scala   |  153 +
 .../planner/plan/stream/sql/SortLimitTest.scala    |  327 ++
 .../table/planner/plan/stream/sql/SortTest.scala   |   79 +
 .../planner/plan/stream/sql/SubplanReuseTest.scala |  300 ++
 .../planner/plan/stream/sql/TableScanTest.scala    |   42 +
 .../planner/plan/stream/sql/TableSourceTest.scala  |  395 +++
 .../table/planner/plan/stream/sql/UnionTest.scala  |   66 +
 .../table/planner/plan/stream/sql/UnnestTest.scala |   26 +
 .../table/planner/plan/stream/sql/ValuesTest.scala |   49 +
 .../plan/stream/sql/agg/AggregateTest.scala        |  263 ++
 .../stream/sql/agg/DistinctAggregateTest.scala     |  224 ++
 .../plan/stream/sql/agg/GroupingSetsTest.scala     |  458 +++
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   52 +
 .../plan/stream/sql/agg/OverAggregateTest.scala    |  433 +++
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   79 +
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  298 ++
 .../plan/stream/sql/join/JoinReorderTest.scala     |   25 +
 .../planner/plan/stream/sql/join/JoinTest.scala    |  284 ++
 .../plan/stream/sql/join/LookupJoinTest.scala      |  493 +++
 .../plan/stream/sql/join/SemiAntiJoinTest.scala    |  576 ++++
 .../plan/stream/sql/join/TemporalJoinTest.scala    |  132 +
 .../plan/stream/sql/join/WindowJoinTest.scala      |  453 +++
 .../planner/plan/stream/table/AggregateTest.scala  |  230 ++
 .../table/planner/plan/stream/table/CalcTest.scala |  161 +
 .../plan/stream/table/ColumnFunctionsTest.scala    |  229 ++
 .../planner/plan/stream/table/CorrelateTest.scala  |  181 ++
 .../plan/stream/table/GroupWindowTest.scala        |  409 +++
 .../table/planner/plan/stream/table/JoinTest.scala |  263 ++
 .../planner/plan/stream/table/OverWindowTest.scala |  222 ++
 .../plan/stream/table/SetOperatorsTest.scala       |   87 +
 .../plan/stream/table/TableSourceTest.scala        |  302 ++
 .../plan/stream/table/TemporalTableJoinTest.scala  |  191 ++
 .../plan/stream/table/TwoStageAggregateTest.scala  |  112 +
 .../stringexpr/AggregateStringExpressionTest.scala |  246 ++
 .../stringexpr/CalcStringExpressionTest.scala      |  183 ++
 .../stringexpr/CorrelateStringExpressionTest.scala |  160 +
 .../GroupWindowStringExpressionTest.scala          |  263 ++
 ...pWindowTableAggregateStringExpressionTest.scala |  227 ++
 .../OverWindowStringExpressionTest.scala           |  248 ++
 .../SetOperatorsStringExpressionTest.scala         |   50 +
 .../table/validation/AggregateValidationTest.scala |  127 +
 .../table/validation/CalcValidationTest.scala      |  165 +
 .../table/validation/CorrelateValidationTest.scala |  178 ++
 .../validation/GroupWindowValidationTest.scala     |  308 ++
 .../validation/OverWindowValidationTest.scala      |  164 +
 .../validation/SetOperatorsValidationTest.scala    |   80 +
 .../table/validation/TableSinkValidationTest.scala |   87 +
 .../TemporalTableJoinValidationTest.scala          |  114 +
 .../validation/UnsupportedOpsValidationTest.scala  |  109 +
 .../plan/trait/FlinkRelDistributionTest.scala      |  165 +
 .../planner/plan/utils/FlinkRelOptUtilTest.scala   |  141 +
 .../planner/plan/utils/FlinkRexUtilTest.scala      |  407 +++
 .../planner/plan/utils/InputTypeBuilder.scala      |   53 +
 .../planner/plan/utils/PartitionPrunerTest.scala   |  111 +
 .../planner/plan/utils/RelDigestUtilTest.scala     |  108 +
 .../planner/plan/utils/RexNodeExtractorTest.scala  |  905 ++++++
 .../planner/plan/utils/RexNodeRewriterTest.scala   |   75 +
 .../table/planner/plan/utils/RexNodeTestBase.scala |   90 +
 .../flink/table/planner/plan/utils/pojos.scala     |   51 +
 .../planner/runtime/batch/sql/CalcITCase.scala     | 1263 ++++++++
 .../runtime/batch/sql/CorrelateITCase.scala        |  369 +++
 .../runtime/batch/sql/CorrelateITCase2.scala       |  267 ++
 .../planner/runtime/batch/sql/DecimalITCase.scala  |  936 ++++++
 .../runtime/batch/sql/Limit0RemoveITCase.scala     |   95 +
 .../planner/runtime/batch/sql/LimitITCase.scala    |  119 +
 .../planner/runtime/batch/sql/MiscITCase.scala     |  616 ++++
 .../runtime/batch/sql/OverWindowITCase.scala       | 2508 +++++++++++++++
 .../batch/sql/PartitionableSinkITCase.scala        |  330 ++
 .../planner/runtime/batch/sql/RankITCase.scala     |  141 +
 .../runtime/batch/sql/SetOperatorsITCase.scala     |  153 +
 .../runtime/batch/sql/SortLimitITCase.scala        |  129 +
 .../runtime/batch/sql/TableScanITCase.scala        |  124 +
 .../runtime/batch/sql/TableSourceITCase.scala      |  208 ++
 .../planner/runtime/batch/sql/UnionITCase.scala    |  140 +
 .../planner/runtime/batch/sql/UnnestITCase.scala   |  254 ++
 .../planner/runtime/batch/sql/ValuesITCase.scala   |   35 +
 .../batch/sql/agg/AggregateITCaseBase.scala        |  885 ++++++
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  208 ++
 .../sql/agg/AggregateReduceGroupingITCase.scala    |  413 +++
 .../batch/sql/agg/AggregateRemoveITCase.scala      |  213 ++
 .../sql/agg/DistinctAggregateITCaseBase.scala      |  312 ++
 .../runtime/batch/sql/agg/GroupingSetsITCase.scala |  564 ++++
 .../runtime/batch/sql/agg/HashAggITCase.scala      |   33 +
 .../sql/agg/HashDistinctAggregateITCase.scala      |   33 +
 .../batch/sql/agg/PruneAggregateCallITCase.scala   |  121 +
 .../runtime/batch/sql/agg/SortAggITCase.scala      |  568 ++++
 .../sql/agg/SortDistinctAggregateITCase.scala      |   97 +
 .../batch/sql/agg/WindowAggregateITCase.scala      |  744 +++++
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  199 ++
 .../sql/join/JoinConditionTypeCoerceITCase.scala   |  175 ++
 .../runtime/batch/sql/join/JoinITCase.scala        |  836 +++++
 .../runtime/batch/sql/join/JoinITCaseHelper.scala  |   53 +
 .../batch/sql/join/JoinWithoutKeyITCase.scala      |  373 +++
 .../runtime/batch/sql/join/LookupJoinITCase.scala  |  234 ++
 .../runtime/batch/sql/join/OuterJoinITCase.scala   |  394 +++
 .../runtime/batch/sql/join/ScalarQueryITCase.scala |   51 +
 .../runtime/batch/sql/join/SemiJoinITCase.scala    |  490 +++
 .../runtime/batch/table/AggregationITCase.scala    |  482 +++
 .../planner/runtime/batch/table/CalcITCase.scala   |  663 ++++
 .../runtime/batch/table/CorrelateITCase.scala      |  385 +++
 .../runtime/batch/table/DecimalITCase.scala        |  791 +++++
 .../runtime/batch/table/GroupWindowITCase.scala    |  342 +++
 .../planner/runtime/batch/table/JoinITCase.scala   |  391 +++
 .../runtime/batch/table/OverWindowITCase.scala     | 1404 +++++++++
 .../runtime/batch/table/SetOperatorsITCase.scala   |  198 ++
 .../planner/runtime/batch/table/SortITCase.scala   |   91 +
 ...AbstractTwoInputStreamOperatorWithTTLTest.scala |  182 ++
 .../planner/runtime/harness/HarnessTestBase.scala  |  122 +
 .../runtime/harness/OverWindowHarnessTest.scala    |  986 ++++++
 .../runtime/stream/sql/AggregateITCase.scala       | 1297 ++++++++
 .../runtime/stream/sql/AggregateRemoveITCase.scala |  254 ++
 .../runtime/stream/sql/AsyncLookupJoinITCase.scala |  352 +++
 .../planner/runtime/stream/sql/CalcITCase.scala    |  263 ++
 .../runtime/stream/sql/CorrelateITCase.scala       |  425 +++
 .../runtime/stream/sql/DeduplicateITCase.scala     |   89 +
 .../planner/runtime/stream/sql/JoinITCase.scala    | 1130 +++++++
 .../runtime/stream/sql/Limit0RemoveITCase.scala    |  187 ++
 .../planner/runtime/stream/sql/LimitITCase.scala   |  109 +
 .../runtime/stream/sql/LookupJoinITCase.scala      |  421 +++
 .../runtime/stream/sql/MatchRecognizeITCase.scala  |  783 +++++
 .../runtime/stream/sql/OverWindowITCase.scala      | 1023 +++++++
 .../stream/sql/PruneAggregateCallITCase.scala      |  130 +
 .../planner/runtime/stream/sql/RankITCase.scala    | 1314 ++++++++
 .../stream/sql/SemiAntiJoinStreamITCase.scala      |  542 ++++
 .../runtime/stream/sql/SetOperatorsITCase.scala    |  134 +
 .../planner/runtime/stream/sql/SortITCase.scala    |  240 ++
 .../runtime/stream/sql/SortLimitITCase.scala       |  105 +
 .../runtime/stream/sql/SplitAggregateITCase.scala  |  334 ++
 .../runtime/stream/sql/TableScanITCase.scala       |  147 +
 .../runtime/stream/sql/TableSourceITCase.scala     |  384 +++
 .../runtime/stream/sql/TemporalJoinITCase.scala    |  168 ++
 .../runtime/stream/sql/TemporalSortITCase.scala    |  163 +
 .../planner/runtime/stream/sql/UnnestITCase.scala  |  325 ++
 .../planner/runtime/stream/sql/ValuesITCase.scala  |   49 +
 .../runtime/stream/sql/WindowAggregateITCase.scala |  266 ++
 .../runtime/stream/sql/WindowJoinITCase.scala      |  948 ++++++
 .../runtime/stream/table/AggregateITCase.scala     |  401 +++
 .../planner/runtime/stream/table/CalcITCase.scala  |  467 +++
 .../runtime/stream/table/CorrelateITCase.scala     |  395 +++
 .../runtime/stream/table/GroupWindowITCase.scala   |  295 ++
 .../planner/runtime/stream/table/JoinITCase.scala  | 1472 +++++++++
 .../stream/table/MiniBatchGroupWindowITCase.scala  |  156 +
 .../runtime/stream/table/OverWindowITCase.scala    |  440 +++
 .../runtime/stream/table/RetractionITCase.scala    |  168 ++
 .../runtime/stream/table/SetOperatorsITCase.scala  |  205 ++
 .../runtime/stream/table/SubQueryITCase.scala      |  148 +
 .../planner/runtime/utils/BatchTableEnvUtil.scala  |  296 ++
 .../planner/runtime/utils/BatchTestBase.scala      |  490 +++
 .../runtime/utils/CollectionBatchExecTable.scala   |  403 +++
 .../utils/InMemoryLookupableTableSource.scala      |  273 ++
 .../planner/runtime/utils/SortTestUtils.scala      |   57 +
 .../planner/runtime/utils/StreamTableEnvUtil.scala |   53 +
 .../planner/runtime/utils/StreamTestSink.scala     |  524 ++++
 .../planner/runtime/utils/StreamingTestBase.scala  |   81 +
 .../runtime/utils/StreamingWithAggTestBase.scala   |   75 +
 .../utils/StreamingWithMiniBatchTestBase.scala     |   74 +
 .../runtime/utils/StreamingWithStateTestBase.scala |  244 ++
 .../table/planner/runtime/utils/TableUtil.scala    |   77 +
 .../table/planner/runtime/utils/TestData.scala     |  476 +++
 .../table/planner/runtime/utils/TestSinkUtil.scala |  121 +
 .../table/planner/runtime/utils/TimeTestUtil.scala |   68 +
 .../utils/UserDefinedFunctionTestUtils.scala       |  438 +++
 .../flink/table/planner/utils/AvgAggFunction.scala |  357 +++
 .../planner/utils/ColumnIntervalUtilTest.scala     |  218 ++
 .../table/planner/utils/CountAggFunction.scala     |   82 +
 .../table/planner/utils/DateTimeTestUtil.scala     |   41 +
 .../planner/utils/LogicalPlanFormatUtils.scala     |   40 +
 .../planner/utils/MemoryTableSourceSinkUtil.scala  |  166 +
 .../flink/table/planner/utils/SumAggFunction.scala |  184 ++
 .../flink/table/planner/utils/TableTestBase.scala  | 1074 +++++++
 .../planner/utils/UserDefinedAggFunctions.scala    |  187 ++
 .../planner/utils/UserDefinedTableFunctions.scala  |  520 ++++
 .../table/planner/utils/testTableSources.scala     |  599 ++++
 .../flink/table/runtime/batch/sql/CalcITCase.scala | 1263 --------
 .../table/runtime/batch/sql/CorrelateITCase.scala  |  369 ---
 .../table/runtime/batch/sql/CorrelateITCase2.scala |  267 --
 .../table/runtime/batch/sql/DecimalITCase.scala    |  936 ------
 .../runtime/batch/sql/Limit0RemoveITCase.scala     |   95 -
 .../table/runtime/batch/sql/LimitITCase.scala      |  119 -
 .../flink/table/runtime/batch/sql/MiscITCase.scala |  616 ----
 .../table/runtime/batch/sql/OverWindowITCase.scala | 2508 ---------------
 .../batch/sql/PartitionableSinkITCase.scala        |  330 --
 .../flink/table/runtime/batch/sql/RankITCase.scala |  141 -
 .../runtime/batch/sql/SetOperatorsITCase.scala     |  153 -
 .../table/runtime/batch/sql/SortLimitITCase.scala  |  129 -
 .../table/runtime/batch/sql/TableScanITCase.scala  |  124 -
 .../runtime/batch/sql/TableSourceITCase.scala      |  208 --
 .../table/runtime/batch/sql/UnionITCase.scala      |  140 -
 .../table/runtime/batch/sql/UnnestITCase.scala     |  256 --
 .../table/runtime/batch/sql/ValuesITCase.scala     |   35 -
 .../batch/sql/agg/AggregateITCaseBase.scala        |  885 ------
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  208 --
 .../sql/agg/AggregateReduceGroupingITCase.scala    |  413 ---
 .../batch/sql/agg/AggregateRemoveITCase.scala      |  213 --
 .../sql/agg/DistinctAggregateITCaseBase.scala      |  312 --
 .../runtime/batch/sql/agg/GroupingSetsITCase.scala |  564 ----
 .../runtime/batch/sql/agg/HashAggITCase.scala      |   33 -
 .../sql/agg/HashDistinctAggregateITCase.scala      |   33 -
 .../batch/sql/agg/PruneAggregateCallITCase.scala   |  121 -
 .../runtime/batch/sql/agg/SortAggITCase.scala      |  568 ----
 .../sql/agg/SortDistinctAggregateITCase.scala      |   97 -
 .../batch/sql/agg/WindowAggregateITCase.scala      |  744 -----
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  199 --
 .../sql/join/JoinConditionTypeCoerceITCase.scala   |  175 --
 .../table/runtime/batch/sql/join/JoinITCase.scala  |  836 -----
 .../runtime/batch/sql/join/JoinITCaseHelper.scala  |   53 -
 .../batch/sql/join/JoinWithoutKeyITCase.scala      |  373 ---
 .../runtime/batch/sql/join/LookupJoinITCase.scala  |  234 --
 .../runtime/batch/sql/join/OuterJoinITCase.scala   |  394 ---
 .../runtime/batch/sql/join/ScalarQueryITCase.scala |   51 -
 .../runtime/batch/sql/join/SemiJoinITCase.scala    |  490 ---
 .../runtime/batch/table/AggregationITCase.scala    |  482 ---
 .../table/runtime/batch/table/CalcITCase.scala     |  663 ----
 .../runtime/batch/table/CorrelateITCase.scala      |  385 ---
 .../table/runtime/batch/table/DecimalITCase.scala  |  791 -----
 .../runtime/batch/table/GroupWindowITCase.scala    |  342 ---
 .../table/runtime/batch/table/JoinITCase.scala     |  391 ---
 .../runtime/batch/table/OverWindowITCase.scala     | 1404 ---------
 .../runtime/batch/table/SetOperatorsITCase.scala   |  198 --
 .../table/runtime/batch/table/SortITCase.scala     |   91 -
 ...AbstractTwoInputStreamOperatorWithTTLTest.scala |  182 --
 .../table/runtime/harness/HarnessTestBase.scala    |  120 -
 .../runtime/harness/OverWindowHarnessTest.scala    |  986 ------
 .../table/runtime/stream/sql/AggregateITCase.scala | 1297 --------
 .../runtime/stream/sql/AggregateRemoveITCase.scala |  254 --
 .../runtime/stream/sql/AsyncLookupJoinITCase.scala |  351 ---
 .../table/runtime/stream/sql/CalcITCase.scala      |  263 --
 .../table/runtime/stream/sql/CorrelateITCase.scala |  425 ---
 .../runtime/stream/sql/DeduplicateITCase.scala     |   89 -
 .../table/runtime/stream/sql/JoinITCase.scala      | 1130 -------
 .../runtime/stream/sql/Limit0RemoveITCase.scala    |  187 --
 .../table/runtime/stream/sql/LimitITCase.scala     |  109 -
 .../runtime/stream/sql/LookupJoinITCase.scala      |  420 ---
 .../runtime/stream/sql/MatchRecognizeITCase.scala  |  782 -----
 .../runtime/stream/sql/OverWindowITCase.scala      | 1023 -------
 .../stream/sql/PruneAggregateCallITCase.scala      |  130 -
 .../table/runtime/stream/sql/RankITCase.scala      | 1314 --------
 .../stream/sql/SemiAntiJoinStreamITCase.scala      |  541 ----
 .../runtime/stream/sql/SetOperatorsITCase.scala    |  134 -
 .../table/runtime/stream/sql/SortITCase.scala      |  240 --
 .../table/runtime/stream/sql/SortLimitITCase.scala |  105 -
 .../runtime/stream/sql/SplitAggregateITCase.scala  |  334 --
 .../table/runtime/stream/sql/TableScanITCase.scala |  146 -
 .../runtime/stream/sql/TableSourceITCase.scala     |  385 ---
 .../runtime/stream/sql/TemporalJoinITCase.scala    |  168 --
 .../runtime/stream/sql/TemporalSortITCase.scala    |  163 -
 .../table/runtime/stream/sql/UnnestITCase.scala    |  325 --
 .../table/runtime/stream/sql/ValuesITCase.scala    |   49 -
 .../runtime/stream/sql/WindowAggregateITCase.scala |  266 --
 .../runtime/stream/sql/WindowJoinITCase.scala      |  948 ------
 .../runtime/stream/table/AggregateITCase.scala     |  401 ---
 .../table/runtime/stream/table/CalcITCase.scala    |  467 ---
 .../runtime/stream/table/CorrelateITCase.scala     |  395 ---
 .../runtime/stream/table/GroupWindowITCase.scala   |  295 --
 .../table/runtime/stream/table/JoinITCase.scala    | 1472 ---------
 .../stream/table/MiniBatchGroupWindowITCase.scala  |  156 -
 .../runtime/stream/table/OverWindowITCase.scala    |  439 ---
 .../runtime/stream/table/RetractionITCase.scala    |  168 --
 .../runtime/stream/table/SetOperatorsITCase.scala  |  205 --
 .../runtime/stream/table/SubQueryITCase.scala      |  148 -
 .../table/runtime/utils/BatchTableEnvUtil.scala    |  296 --
 .../flink/table/runtime/utils/BatchTestBase.scala  |  490 ---
 .../runtime/utils/CollectionBatchExecTable.scala   |  403 ---
 .../utils/InMemoryLookupableTableSource.scala      |  273 --
 .../flink/table/runtime/utils/SortTestUtils.scala  |   57 -
 .../table/runtime/utils/StreamTableEnvUtil.scala   |   53 -
 .../flink/table/runtime/utils/StreamTestSink.scala |  524 ----
 .../table/runtime/utils/StreamingTestBase.scala    |   81 -
 .../runtime/utils/StreamingWithAggTestBase.scala   |   74 -
 .../utils/StreamingWithMiniBatchTestBase.scala     |   73 -
 .../runtime/utils/StreamingWithStateTestBase.scala |  244 --
 .../flink/table/runtime/utils/TableUtil.scala      |   77 -
 .../flink/table/runtime/utils/TestData.scala       |  476 ---
 .../flink/table/runtime/utils/TestSinkUtil.scala   |  121 -
 .../flink/table/runtime/utils/TimeTestUtil.scala   |   68 -
 .../utils/UserDefinedFunctionTestUtils.scala       |  438 ---
 .../apache/flink/table/util/AvgAggFunction.scala   |  357 ---
 .../flink/table/util/ColumnIntervalUtilTest.scala  |  219 --
 .../apache/flink/table/util/CountAggFunction.scala |   82 -
 .../apache/flink/table/util/DateTimeTestUtil.scala |   41 -
 .../flink/table/util/LogicalPlanFormatUtils.scala  |   40 -
 .../table/util/MemoryTableSourceSinkUtil.scala     |  165 -
 .../apache/flink/table/util/SumAggFunction.scala   |  184 --
 .../apache/flink/table/util/TableTestBase.scala    | 1072 -------
 .../flink/table/util/UserDefinedAggFunctions.scala |  187 --
 .../table/util/UserDefinedTableFunctions.scala     |  520 ----
 .../apache/flink/table/util/testTableSources.scala |  599 ----
 .../expressions/PlannerExpressionParserImpl.scala  |    3 +
 2029 files changed, 213832 insertions(+), 213676 deletions(-)

diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java
index e3fc7cc..29c9227 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java
@@ -170,8 +170,8 @@ public class EnvironmentSettings {
 		 * enabled.
 		 */
 		public Builder useBlinkPlanner() {
-			this.plannerClass = "org.apache.flink.table.planner.BlinkPlannerFactory";
-			this.executorClass = "org.apache.flink.table.executor.BlinkExecutorFactory";
+			this.plannerClass = "org.apache.flink.table.planner.delegation.BlinkPlannerFactory";
+			this.executorClass = "org.apache.flink.table.planner.delegation.BlinkExecutorFactory";
 			return this;
 		}
 
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index dedddfb..e366f52 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -16,7 +16,7 @@
  */
 package org.apache.calcite.sql2rel;
 
-import org.apache.flink.table.plan.rules.logical.FlinkFilterJoinRule;
+import org.apache.flink.table.planner.plan.rules.logical.FlinkFilterJoinRule;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -63,7 +63,6 @@ import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.FilterCorrelateRule;
-import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/FlinkCalciteCatalogReader.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/FlinkCalciteCatalogReader.java
deleted file mode 100644
index c83f629..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/FlinkCalciteCatalogReader.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.calcite;
-
-import org.apache.flink.table.plan.schema.FlinkRelOptTable;
-import org.apache.flink.table.plan.schema.FlinkTable;
-
-import org.apache.calcite.config.CalciteConnectionConfig;
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.sql.validate.SqlNameMatchers;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/**
- * Flink specific {@link CalciteCatalogReader} that changes the RelOptTable which wrapped a
- * FlinkTable to a {@link FlinkRelOptTable}.
- */
-public class FlinkCalciteCatalogReader extends CalciteCatalogReader {
-
-	public FlinkCalciteCatalogReader(
-		CalciteSchema rootSchema,
-		List<List<String>> defaultSchemas,
-		RelDataTypeFactory typeFactory,
-		CalciteConnectionConfig config) {
-
-		super(
-			rootSchema,
-			SqlNameMatchers.withCaseSensitive(config != null && config.caseSensitive()),
-				Stream.concat(
-					defaultSchemas.stream(),
-					Stream.of(Collections.<String>emptyList())
-				).collect(Collectors.toList()),
-			typeFactory,
-			config);
-	}
-
-	@Override
-	public Prepare.PreparingTable getTable(List<String> names) {
-		Prepare.PreparingTable originRelOptTable = super.getTable(names);
-		if (originRelOptTable == null) {
-			return null;
-		} else {
-			// Wrap FlinkTable as FlinkRelOptTable to use in query optimization.
-			FlinkTable table = originRelOptTable.unwrap(FlinkTable.class);
-			if (table != null) {
-				return FlinkRelOptTable.create(
-					originRelOptTable.getRelOptSchema(),
-					originRelOptTable.getRowType(),
-					originRelOptTable.getQualifiedName(),
-					table);
-			} else {
-				return originRelOptTable;
-			}
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/FlinkReturnTypes.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/FlinkReturnTypes.java
deleted file mode 100644
index e1f3199..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/FlinkReturnTypes.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.calcite.type;
-
-import org.apache.flink.table.calcite.FlinkTypeFactory;
-import org.apache.flink.table.calcite.FlinkTypeSystem;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.MapType;
-import org.apache.flink.table.types.logical.VarCharType;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlOperatorBinding;
-import org.apache.calcite.sql.type.OrdinalReturnTypeInference;
-import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.type.SqlTypeTransforms;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-
-import java.math.BigDecimal;
-
-/**
- * Type inference in Flink.
- */
-public class FlinkReturnTypes {
-
-	/**
-	 * ROUND(num [,len]) type inference.
-	 */
-	public static final SqlReturnTypeInference ROUND_FUNCTION = new SqlReturnTypeInference() {
-		@Override
-		public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-			final RelDataType numType = opBinding.getOperandType(0);
-			if (numType.getSqlTypeName() != SqlTypeName.DECIMAL) {
-				return numType;
-			}
-			final BigDecimal lenVal;
-			if (opBinding.getOperandCount() == 1) {
-				lenVal = BigDecimal.ZERO;
-			} else if (opBinding.getOperandCount() == 2) {
-				lenVal = getArg1Literal(opBinding); // may return null
-			} else {
-				throw new AssertionError();
-			}
-			if (lenVal == null) {
-				return numType; //
-			}
-			// ROUND( decimal(p,s), r )
-			final int p = numType.getPrecision();
-			final int s = numType.getScale();
-			final int r = lenVal.intValueExact();
-			DecimalType dt = FlinkTypeSystem.inferRoundType(p, s, r);
-			return opBinding.getTypeFactory().createSqlType(
-				SqlTypeName.DECIMAL, dt.getPrecision(), dt.getScale());
-		}
-
-		private BigDecimal getArg1Literal(SqlOperatorBinding opBinding) {
-			try {
-				return opBinding.getOperandLiteralValue(1, BigDecimal.class);
-			} catch (Throwable e) {
-				return null;
-			}
-		}
-	};
-
-	/**
-	 * Type-inference strategy whereby the result type of a call is the type of
-	 * the operand #0 (0-based), with nulls always allowed.
-	 */
-	public static final SqlReturnTypeInference ARG0_VARCHAR_FORCE_NULLABLE = new OrdinalReturnTypeInference(0) {
-		@Override
-		public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-			RelDataType type = super.inferReturnType(opBinding);
-			RelDataType newType;
-			switch (type.getSqlTypeName()) {
-				case CHAR:
-					newType = opBinding.getTypeFactory().createSqlType(SqlTypeName.VARCHAR, type.getPrecision());
-					break;
-				case VARCHAR:
-					newType = type;
-					break;
-				default:
-					throw new UnsupportedOperationException("Unsupported type: " + type);
-			}
-			return opBinding.getTypeFactory().createTypeWithNullability(newType, true);
-		}
-	};
-
-	public static final SqlReturnTypeInference FLINK_QUOTIENT_NULLABLE = opBinding -> {
-		RelDataType type1 = opBinding.getOperandType(0);
-		RelDataType type2 = opBinding.getOperandType(1);
-		if (SqlTypeUtil.isDecimal(type1) || SqlTypeUtil.isDecimal(type2)) {
-			return ReturnTypes.QUOTIENT_NULLABLE.inferReturnType(opBinding);
-		} else {
-			RelDataType doubleType = opBinding.getTypeFactory().createSqlType(SqlTypeName.DOUBLE);
-			if (type1.isNullable() || type2.isNullable()) {
-				return opBinding.getTypeFactory().createTypeWithNullability(doubleType, true);
-			} else {
-				return doubleType;
-			}
-		}
-	};
-
-	public static final SqlReturnTypeInference FLINK_DIV_NULLABLE = opBinding -> {
-		RelDataType type1 = opBinding.getOperandType(0);
-		RelDataType type2 = opBinding.getOperandType(1);
-		RelDataType returnType;
-		if (SqlTypeUtil.isDecimal(type1) || SqlTypeUtil.isDecimal(type2)) {
-			DecimalType dt = FlinkTypeSystem.inferIntDivType(
-					type1.getPrecision(), type1.getScale(), type2.getScale());
-			returnType = opBinding.getTypeFactory().createSqlType(
-				SqlTypeName.DECIMAL, dt.getPrecision(), dt.getScale());
-		} else { // both are primitive
-			returnType = type1;
-		}
-		return opBinding.getTypeFactory().createTypeWithNullability(returnType,
-			type1.isNullable() || type2.isNullable());
-	};
-
-	/**
-	 * Type-inference strategy that always returns "VARCHAR(2000)" with nulls always allowed.
-	 */
-	public static final SqlReturnTypeInference VARCHAR_2000_NULLABLE =
-		ReturnTypes.cascade(ReturnTypes.VARCHAR_2000, SqlTypeTransforms.FORCE_NULLABLE);
-
-	public static final SqlReturnTypeInference ROUND_FUNCTION_NULLABLE =
-		ReturnTypes.cascade(ROUND_FUNCTION, SqlTypeTransforms.TO_NULLABLE);
-
-	public static final SqlReturnTypeInference NUMERIC_FROM_ARG1_DEFAULT1 =
-		new NumericOrDefaultReturnTypeInference(1, 1);
-
-	public static final SqlReturnTypeInference NUMERIC_FROM_ARG1_DEFAULT1_NULLABLE =
-		ReturnTypes.cascade(NUMERIC_FROM_ARG1_DEFAULT1, SqlTypeTransforms.TO_NULLABLE);
-
-	public static final SqlReturnTypeInference STR_MAP_NULLABLE = ReturnTypes.explicit(
-			factory -> ((FlinkTypeFactory) factory).createFieldTypeFromLogicalType(
-		new MapType(new VarCharType(VarCharType.MAX_LENGTH), new VarCharType(VarCharType.MAX_LENGTH))));
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericExceptFirstOperandChecker.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericExceptFirstOperandChecker.java
deleted file mode 100644
index 3e5e146..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericExceptFirstOperandChecker.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.calcite.type;
-
-import org.apache.calcite.sql.SqlCallBinding;
-import org.apache.calcite.sql.SqlOperandCountRange;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Parameter type-checking strategy where all operand types except first one must be numeric type.
- */
-public class NumericExceptFirstOperandChecker implements SqlOperandTypeChecker {
-
-	private int nOperands;
-
-	public NumericExceptFirstOperandChecker(int nOperands) {
-		this.nOperands = nOperands;
-	}
-
-	@Override
-	public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
-		for (int i = 1; i < callBinding.getOperandCount(); i++) {
-			if (!SqlTypeUtil.isNumeric(callBinding.getOperandType(i))) {
-				if (!throwOnFailure) {
-					return false;
-				}
-				throw callBinding.newValidationSignatureError();
-			}
-		}
-		return true;
-	}
-
-	@Override
-	public SqlOperandCountRange getOperandCountRange() {
-		if (nOperands == -1) {
-			return SqlOperandCountRanges.any();
-		} else {
-			return SqlOperandCountRanges.of(nOperands);
-		}
-	}
-
-	@Override
-	public String getAllowedSignatures(SqlOperator op, String opName) {
-		final String anyType = "ANY_TYPE";
-		final String numericType = "NUMERIC_TYPE";
-
-		if (nOperands == -1) {
-			return SqlUtil.getAliasedSignature(op, opName,
-				Arrays.asList(anyType, numericType, "..."));
-		} else {
-			List<String> types = new ArrayList<>();
-			types.add(anyType);
-			types.addAll(Collections.nCopies(nOperands - 1, numericType));
-			return SqlUtil.getAliasedSignature(op, opName, types);
-		}
-	}
-
-	@Override
-	public Consistency getConsistency() {
-		return Consistency.NONE;
-	}
-
-	@Override
-	public boolean isOptional(int i) {
-		return false;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericOrDefaultReturnTypeInference.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericOrDefaultReturnTypeInference.java
deleted file mode 100644
index 290fdeb..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/NumericOrDefaultReturnTypeInference.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.calcite.type;
-
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlOperatorBinding;
-import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeUtil;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Determine the return type of functions with numeric arguments.
- * The return type is the type of the argument with the largest range.
- * We start to consider the arguments from the `startTypeIdx`-th one.
- * If one of the arguments is not of numeric type,
- * we return the type of the `defaultTypeIdx`-th argument instead.
- */
-public class NumericOrDefaultReturnTypeInference implements SqlReturnTypeInference {
-	// Default argument whose type is returned
-	// when one of the arguments from the `startTypeIdx`-th isn't of numeric type.
-	private int defaultTypeIdx;
-	// We check from the `startTypeIdx`-th argument that
-	// if all the following arguments are of numeric type.
-	// Previous arguments are ignored.
-	private int startTypeIdx;
-
-	public NumericOrDefaultReturnTypeInference(int defaultTypeIdx) {
-		this(defaultTypeIdx, 0);
-	}
-
-	public NumericOrDefaultReturnTypeInference(int defaultTypeIdx, int startTypeIdx) {
-		this.defaultTypeIdx = defaultTypeIdx;
-		this.startTypeIdx = startTypeIdx;
-	}
-
-	@Override
-	public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-		int nOperands = opBinding.getOperandCount();
-		List<RelDataType> types = new ArrayList<>();
-		for (int i = startTypeIdx; i < nOperands; i++) {
-			RelDataType type = opBinding.getOperandType(i);
-			if (SqlTypeUtil.isNumeric(type)) {
-				types.add(type);
-			} else {
-				return opBinding.getOperandType(defaultTypeIdx);
-			}
-		}
-		return opBinding.getTypeFactory().leastRestrictive(types);
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/RepeatFamilyOperandTypeChecker.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/RepeatFamilyOperandTypeChecker.java
deleted file mode 100644
index e9cac2b..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/calcite/type/RepeatFamilyOperandTypeChecker.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.calcite.type;
-
-import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlCallBinding;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperandCountRange;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlUtil;
-import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
-import org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-import java.util.Arrays;
-
-import static org.apache.calcite.util.Static.RESOURCE;
-
-
-/**
- * Parameter type-checking strategy where there must be more than one operands,
- * and all operands must have a same specific SqlTypeFamily type.
- */
-public class RepeatFamilyOperandTypeChecker implements SqlOperandTypeChecker {
-
-	protected final SqlTypeFamily family;
-
-	public RepeatFamilyOperandTypeChecker(SqlTypeFamily family) {
-		this.family = family;
-	}
-
-	public boolean checkSingleOperandType(
-		SqlCallBinding callBinding,
-		SqlNode node,
-		boolean throwOnFailure) {
-
-		if (SqlUtil.isNullLiteral(node, false)) {
-			if (throwOnFailure) {
-				throw callBinding.getValidator().newValidationError(node,
-					RESOURCE.nullIllegal());
-			} else {
-				return false;
-			}
-		}
-
-		RelDataType type = callBinding.getValidator().deriveType(
-			callBinding.getScope(),
-			node);
-		SqlTypeName typeName = type.getSqlTypeName();
-
-		// Pass type checking for operators if it's of type 'ANY'.
-		if (typeName.getFamily() == SqlTypeFamily.ANY) {
-			return true;
-		}
-
-		if (!family.getTypeNames().contains(typeName)) {
-			if (throwOnFailure) {
-				throw callBinding.newValidationSignatureError();
-			}
-			return false;
-		}
-		return true;
-	}
-
-	public boolean checkOperandTypes(
-		SqlCallBinding callBinding,
-		boolean throwOnFailure) {
-
-		for (Ord<SqlNode> op : Ord.zip(callBinding.operands())) {
-			if (!checkSingleOperandType(
-				callBinding,
-				op.e,
-				false)) {
-				// TODO: check type coercion when we support implicit type conversion
-				// recheck to validate.
-				for (Ord<SqlNode> op1 : Ord.zip(callBinding.operands())) {
-					if (!checkSingleOperandType(
-						callBinding,
-						op1.e,
-						throwOnFailure)) {
-						return false;
-					}
-				}
-				return false;
-			}
-		}
-		return true;
-	}
-
-	public SqlOperandCountRange getOperandCountRange() {
-		return SqlOperandCountRanges.from(1);
-	}
-
-	public String getAllowedSignatures(SqlOperator op, String opName) {
-		return SqlUtil.getAliasedSignature(op, opName, Arrays.asList(family.toString(), "..."));
-	}
-
-	public Consistency getConsistency() {
-		return Consistency.NONE;
-	}
-
-	public boolean isOptional(int i) {
-		return false;
-	}
-}
-
-// End RepeatFamilyOperandTypeChecker.java
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java
deleted file mode 100644
index 678eca3..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.Table;
-
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * A mapping between Flink's catalog and Calcite's schema. This enables to look up and access objects(tables, views,
- * functions, types) in SQL queries without registering them in advance. Databases are registered as sub-schemas
- * in the schema.
- */
-@Internal
-public class CatalogCalciteSchema extends FlinkSchema {
-
-	private final String catalogName;
-	private final Catalog catalog;
-	// Flag that tells if the current planner should work in a batch or streaming mode.
-	private final boolean isStreamingMode;
-
-	public CatalogCalciteSchema(String catalogName, Catalog catalog, boolean isStreamingMode) {
-		this.catalogName = catalogName;
-		this.catalog = catalog;
-		this.isStreamingMode = isStreamingMode;
-	}
-
-	/**
-	 * Look up a sub-schema (database) by the given sub-schema name.
-	 *
-	 * @param schemaName name of sub-schema to look up
-	 * @return the sub-schema with a given database name, or null
-	 */
-	@Override
-	public Schema getSubSchema(String schemaName) {
-		if (catalog.databaseExists(schemaName)) {
-			return new DatabaseCalciteSchema(schemaName, catalogName, catalog, isStreamingMode);
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public Set<String> getSubSchemaNames() {
-		return new HashSet<>(catalog.listDatabases());
-	}
-
-	@Override
-	public Table getTable(String name) {
-		return null;
-	}
-
-	@Override
-	public Set<String> getTableNames() {
-		return new HashSet<>();
-	}
-
-	@Override
-	public Expression getExpression(SchemaPlus parentSchema, String name) {
-		return  Schemas.subSchemaExpression(parentSchema, name, getClass());
-	}
-
-	@Override
-	public boolean isMutable() {
-		return true;
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java
deleted file mode 100644
index 7ca0a41..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.api.TableException;
-
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Table;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Bridge between the {@link CatalogManager} and the {@link Schema}. This way we can query Flink's specific catalogs
- * from Calcite.
- *
- * <p>The mapping for {@link Catalog}s is modeled as a strict two-level reference structure for Flink in Calcite,
- * the full path of objects is of format [catalog_name].[db_name].[meta-object_name].
- *
- * <p>It also supports {@link ExternalCatalog}s. An external catalog maps 1:1 to the Calcite's schema.
- */
-@Internal
-public class CatalogManagerCalciteSchema extends FlinkSchema {
-
-	private final CatalogManager catalogManager;
-	// Flag that tells if the current planner should work in a batch or streaming mode.
-	private final boolean isStreamingMode;
-
-	public CatalogManagerCalciteSchema(CatalogManager catalogManager, boolean isStreamingMode) {
-		this.catalogManager = catalogManager;
-		this.isStreamingMode = isStreamingMode;
-	}
-
-	@Override
-	public Table getTable(String name) {
-		return null;
-	}
-
-	@Override
-	public Set<String> getTableNames() {
-		return Collections.emptySet();
-	}
-
-	@Override
-	public Schema getSubSchema(String name) {
-		Schema schema = catalogManager.getCatalog(name)
-			.map(catalog -> new CatalogCalciteSchema(name, catalog, isStreamingMode))
-			.orElse(null);
-
-		if (schema == null && catalogManager.getExternalCatalog(name).isPresent()) {
-			throw new TableException("ExternalCatalog is deprecated and is not supported in blink planner");
-		}
-		return schema;
-	}
-
-	@Override
-	public Set<String> getSubSchemaNames() {
-		return new HashSet<>(catalogManager.getCatalogs());
-	}
-
-	@Override
-	public Expression getExpression(SchemaPlus parentSchema, String name) {
-		return null;
-	}
-
-	@Override
-	public boolean isMutable() {
-		return false;
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java
deleted file mode 100644
index 60c773d..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.catalog.exceptions.CatalogException;
-import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
-import org.apache.flink.table.catalog.exceptions.TableNotExistException;
-import org.apache.flink.table.factories.TableFactory;
-import org.apache.flink.table.factories.TableFactoryUtil;
-import org.apache.flink.table.factories.TableSourceFactory;
-import org.apache.flink.table.operations.DataStreamQueryOperation;
-import org.apache.flink.table.operations.QueryOperation;
-import org.apache.flink.table.operations.RichTableSourceQueryOperation;
-import org.apache.flink.table.plan.schema.TableSinkTable;
-import org.apache.flink.table.plan.schema.TableSourceTable;
-import org.apache.flink.table.plan.stats.FlinkStatistic;
-import org.apache.flink.table.sources.LookupableTableSource;
-import org.apache.flink.table.sources.StreamTableSource;
-import org.apache.flink.table.sources.TableSource;
-
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Schemas;
-import org.apache.calcite.schema.Table;
-
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.Set;
-
-import static java.lang.String.format;
-
-/**
- * A mapping between Flink catalog's database and Calcite's schema.
- * Tables are registered as tables in the schema.
- */
-class DatabaseCalciteSchema extends FlinkSchema {
-	private final String databaseName;
-	private final String catalogName;
-	private final Catalog catalog;
-	// Flag that tells if the current planner should work in a batch or streaming mode.
-	private final boolean isStreamingMode;
-
-	public DatabaseCalciteSchema(String databaseName, String catalogName, Catalog catalog, boolean isStreamingMode) {
-		this.databaseName = databaseName;
-		this.catalogName = catalogName;
-		this.catalog = catalog;
-		this.isStreamingMode = isStreamingMode;
-	}
-
-	@Override
-	public Table getTable(String tableName) {
-		ObjectPath tablePath = new ObjectPath(databaseName, tableName);
-
-		try {
-			if (!catalog.tableExists(tablePath)) {
-				return null;
-			}
-
-			CatalogBaseTable table = catalog.getTable(tablePath);
-
-			// TODO supports GenericCatalogView
-			if (table instanceof QueryOperationCatalogView) {
-				QueryOperationCatalogView view = (QueryOperationCatalogView) table;
-				QueryOperation operation = view.getQueryOperation();
-				if (operation instanceof DataStreamQueryOperation) {
-					List<String> qualifiedName = Arrays.asList(catalogName, databaseName, tableName);
-					((DataStreamQueryOperation) operation).setQualifiedName(qualifiedName);
-				} else if (operation instanceof RichTableSourceQueryOperation) {
-					List<String> qualifiedName = Arrays.asList(catalogName, databaseName, tableName);
-					((RichTableSourceQueryOperation) operation).setQualifiedName(qualifiedName);
-				}
-				return QueryOperationCatalogViewTable.createCalciteTable(view);
-			} else if (table instanceof ConnectorCatalogTable) {
-				return convertConnectorTable((ConnectorCatalogTable<?, ?>) table);
-			} else if (table instanceof CatalogTable) {
-				return convertCatalogTable(tablePath, (CatalogTable) table);
-			} else {
-				throw new TableException("Unsupported table type: " + table);
-			}
-		} catch (TableNotExistException | CatalogException e) {
-			// TableNotExistException should never happen, because we are checking it exists
-			// via catalog.tableExists
-			throw new TableException(format(
-					"A failure occurred when accessing table. Table path [%s, %s, %s]",
-					catalogName,
-					databaseName,
-					tableName), e);
-		}
-	}
-
-	private Table convertConnectorTable(ConnectorCatalogTable<?, ?> table) {
-		Optional<TableSourceTable> tableSourceTable = table.getTableSource()
-			.map(tableSource -> {
-				if (!(tableSource instanceof StreamTableSource ||
-					tableSource instanceof LookupableTableSource)) {
-					throw new TableException(
-						"Only StreamTableSource and LookupableTableSource can be used in Blink planner.");
-				}
-				if (!isStreamingMode && tableSource instanceof StreamTableSource &&
-					!((StreamTableSource<?>) tableSource).isBounded()) {
-					throw new TableException("Only bounded StreamTableSource can be used in batch mode.");
-				}
-				return new TableSourceTable<>(
-					tableSource,
-					isStreamingMode,
-					FlinkStatistic.UNKNOWN());
-			});
-		if (tableSourceTable.isPresent()) {
-			return tableSourceTable.get();
-		} else {
-			Optional<TableSinkTable> tableSinkTable = table.getTableSink()
-				.map(tableSink -> new TableSinkTable<>(
-					tableSink,
-					FlinkStatistic.UNKNOWN()));
-			if (tableSinkTable.isPresent()) {
-				return tableSinkTable.get();
-			} else {
-				throw new TableException("Cannot convert a connector table " +
-					"without either source or sink.");
-			}
-		}
-	}
-
-	private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) {
-		TableSource<?> tableSource;
-		Optional<TableFactory> tableFactory = catalog.getTableFactory();
-		if (tableFactory.isPresent()) {
-			TableFactory tf = tableFactory.get();
-			if (tf instanceof TableSourceFactory) {
-				tableSource = ((TableSourceFactory) tf).createTableSource(tablePath, table);
-			} else {
-				throw new TableException(String.format("Cannot query a sink-only table. TableFactory provided by catalog %s must implement TableSourceFactory",
-					catalog.getClass()));
-			}
-		} else {
-			tableSource = TableFactoryUtil.findAndCreateTableSource(table);
-		}
-
-		if (!(tableSource instanceof StreamTableSource)) {
-			throw new TableException("Catalog tables support only StreamTableSource and InputFormatTableSource");
-		}
-
-		return new TableSourceTable<>(
-			tableSource,
-			!((StreamTableSource<?>) tableSource).isBounded(),
-			FlinkStatistic.UNKNOWN()
-		);
-	}
-
-	@Override
-	public Set<String> getTableNames() {
-		try {
-			return new HashSet<>(catalog.listTables(databaseName));
-		} catch (DatabaseNotExistException e) {
-			throw new CatalogException(e);
-		}
-	}
-
-	@Override
-	public Schema getSubSchema(String s) {
-		return null;
-	}
-
-	@Override
-	public Set<String> getSubSchemaNames() {
-		return new HashSet<>();
-	}
-
-	@Override
-	public Expression getExpression(SchemaPlus parentSchema, String name) {
-		return Schemas.subSchemaExpression(parentSchema, name, getClass());
-	}
-
-	@Override
-	public boolean isMutable() {
-		return true;
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FlinkSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FlinkSchema.java
deleted file mode 100644
index c849f4b1..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FlinkSchema.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaVersion;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-
-/**
- * Base class for flink {@link Schema}, which provides some default implementations.
- */
-public abstract class FlinkSchema implements Schema {
-
-	@Override
-	public RelProtoDataType getType(String name) {
-		return null;
-	}
-
-	@Override
-	public Set<String> getTypeNames() {
-		return Collections.emptySet();
-	}
-
-	@Override
-	public Collection<Function> getFunctions(String name) {
-		return Collections.emptyList();
-	}
-
-	@Override
-	public Set<String> getFunctionNames() {
-		return Collections.emptySet();
-	}
-
-	@Override
-	public Schema snapshot(SchemaVersion version) {
-		return this;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java
deleted file mode 100644
index dc5b918..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/FunctionCatalogOperatorTable.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.calcite.FlinkTypeFactory;
-import org.apache.flink.table.functions.AggregateFunctionDefinition;
-import org.apache.flink.table.functions.FunctionDefinition;
-import org.apache.flink.table.functions.ScalarFunctionDefinition;
-import org.apache.flink.table.functions.TableFunctionDefinition;
-import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils;
-import org.apache.flink.table.types.utils.TypeConversions;
-
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.validate.SqlNameMatcher;
-
-import java.util.List;
-import java.util.Optional;
-
-/**
- * Thin adapter between {@link SqlOperatorTable} and {@link FunctionCatalog}.
- */
-@Internal
-public class FunctionCatalogOperatorTable implements SqlOperatorTable {
-
-	private final FunctionCatalog functionCatalog;
-	private final FlinkTypeFactory typeFactory;
-
-	public FunctionCatalogOperatorTable(
-			FunctionCatalog functionCatalog,
-			FlinkTypeFactory typeFactory) {
-		this.functionCatalog = functionCatalog;
-		this.typeFactory = typeFactory;
-	}
-
-	@Override
-	public void lookupOperatorOverloads(
-			SqlIdentifier opName,
-			SqlFunctionCategory category,
-			SqlSyntax syntax,
-			List<SqlOperator> operatorList,
-			SqlNameMatcher nameMatcher) {
-		if (!opName.isSimple()) {
-			return;
-		}
-
-		// We lookup only user functions via CatalogOperatorTable. Built in functions should
-		// go through BasicOperatorTable
-		if (isNotUserFunction(category)) {
-			return;
-		}
-
-		String name = opName.getSimple();
-		Optional<FunctionLookup.Result> candidateFunction = functionCatalog.lookupFunction(name);
-
-		candidateFunction.flatMap(lookupResult ->
-			convertToSqlFunction(category, name, lookupResult.getFunctionDefinition())
-		).ifPresent(operatorList::add);
-	}
-
-	private boolean isNotUserFunction(SqlFunctionCategory category) {
-		return category != null && !category.isUserDefinedNotSpecificFunction();
-	}
-
-	private Optional<SqlFunction> convertToSqlFunction(
-			SqlFunctionCategory category,
-			String name,
-			FunctionDefinition functionDefinition) {
-		if (functionDefinition instanceof AggregateFunctionDefinition) {
-			return convertAggregateFunction(name, (AggregateFunctionDefinition) functionDefinition);
-		} else if (functionDefinition instanceof ScalarFunctionDefinition) {
-			return convertScalarFunction(name, (ScalarFunctionDefinition) functionDefinition);
-		} else if (functionDefinition instanceof TableFunctionDefinition &&
-				category != null &&
-				category.isTableFunction()) {
-			return convertTableFunction(name, (TableFunctionDefinition) functionDefinition);
-		}
-
-		return Optional.empty();
-	}
-
-	private Optional<SqlFunction> convertAggregateFunction(
-			String name,
-			AggregateFunctionDefinition functionDefinition) {
-		SqlFunction aggregateFunction = UserDefinedFunctionUtils.createAggregateSqlFunction(
-			name,
-			name,
-			functionDefinition.getAggregateFunction(),
-			TypeConversions.fromLegacyInfoToDataType(functionDefinition.getResultTypeInfo()),
-			TypeConversions.fromLegacyInfoToDataType(functionDefinition.getAccumulatorTypeInfo()),
-			typeFactory
-		);
-		return Optional.of(aggregateFunction);
-	}
-
-	private Optional<SqlFunction> convertScalarFunction(String name, ScalarFunctionDefinition functionDefinition) {
-		SqlFunction scalarFunction = UserDefinedFunctionUtils.createScalarSqlFunction(
-			name,
-			name,
-			functionDefinition.getScalarFunction(),
-			typeFactory
-		);
-		return Optional.of(scalarFunction);
-	}
-
-	private Optional<SqlFunction> convertTableFunction(String name, TableFunctionDefinition functionDefinition) {
-		SqlFunction tableFunction = UserDefinedFunctionUtils.createTableSqlFunction(
-			name,
-			name,
-			functionDefinition.getTableFunction(),
-			TypeConversions.fromLegacyInfoToDataType(functionDefinition.getResultType()),
-			typeFactory
-		);
-		return Optional.of(tableFunction);
-	}
-
-	@Override
-	public List<SqlOperator> getOperatorList() {
-		throw new UnsupportedOperationException("This should never be called");
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
deleted file mode 100644
index 06b0eaa..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/catalog/QueryOperationCatalogViewTable.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.catalog;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.table.api.TableSchema;
-import org.apache.flink.table.calcite.FlinkRelBuilder;
-import org.apache.flink.table.calcite.FlinkTypeFactory;
-import org.apache.flink.table.plan.schema.FlinkTable;
-import org.apache.flink.table.plan.stats.FlinkStatistic;
-import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.util.JavaScalaConversionUtil;
-
-import org.apache.calcite.plan.RelOptTable;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.calcite.schema.TranslatableTable;
-
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Collectors;
-
-/**
- * A bridge between a Flink's specific {@link QueryOperationCatalogView} and a Calcite's
- * {@link org.apache.calcite.schema.Table}. It implements {@link TranslatableTable} interface. This enables
- * direct translation from {@link org.apache.flink.table.operations.QueryOperation} to {@link RelNode}.
- *
- * <p>NOTE: Due to legacy inconsistency in null handling in the {@link TableSchema} the translation might introduce
- * additional cast to comply with manifested schema in
- * {@link QueryOperationCatalogViewTable#getRowType(RelDataTypeFactory)}.
- */
-@Internal
-public class QueryOperationCatalogViewTable extends FlinkTable implements TranslatableTable {
-	private final QueryOperationCatalogView catalogView;
-	private final RelProtoDataType rowType;
-	private final FlinkStatistic statistic;
-
-	public static QueryOperationCatalogViewTable createCalciteTable(QueryOperationCatalogView catalogView) {
-		return new QueryOperationCatalogViewTable(catalogView, typeFactory -> {
-			TableSchema tableSchema = catalogView.getSchema();
-			List<String> fieldNames = Arrays.asList(tableSchema.getFieldNames());
-			List<LogicalType> fieldTypes = Arrays.stream(tableSchema.getFieldDataTypes()).map(
-					LogicalTypeDataTypeConverter::fromDataTypeToLogicalType).collect(Collectors.toList());
-			return ((FlinkTypeFactory) typeFactory).buildRelNodeRowType(
-					JavaScalaConversionUtil.toScala(fieldNames),
-					JavaScalaConversionUtil.toScala(fieldTypes));
-		}, FlinkStatistic.UNKNOWN()); // TODO supports statistic
-	}
-
-	private QueryOperationCatalogViewTable(
-			QueryOperationCatalogView catalogView,
-			RelProtoDataType rowType,
-			FlinkStatistic statistic) {
-		this.catalogView = catalogView;
-		this.rowType = rowType;
-		this.statistic = statistic;
-	}
-
-	@Override
-	public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable relOptTable) {
-		FlinkRelBuilder relBuilder = FlinkRelBuilder.of(context.getCluster(), relOptTable);
-
-		RelNode relNode = relBuilder.queryOperation(catalogView.getQueryOperation()).build();
-		return RelOptUtil.createCastRel(relNode, rowType.apply(relBuilder.getTypeFactory()), false);
-	}
-
-	@Override
-	public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-		return rowType.apply(typeFactory);
-	}
-
-	public QueryOperationCatalogView getCatalogView() {
-		return catalogView;
-	}
-
-	@Override
-	public FlinkStatistic getStatistic() {
-		return statistic;
-	}
-
-	@Override
-	public FlinkTable copy(FlinkStatistic statistic) {
-		return new QueryOperationCatalogViewTable(catalogView, rowType, statistic);
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
deleted file mode 100644
index 7d6e450..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BatchExecutor.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.executor;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.InputDependencyConstraint;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.common.operators.ResourceSpec;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.runtime.jobgraph.ScheduleMode;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.transformations.ShuffleMode;
-import org.apache.flink.table.api.config.ExecutionConfigOptions;
-import org.apache.flink.table.delegation.Executor;
-import org.apache.flink.table.plan.nodes.resource.NodeResourceUtil;
-
-import java.util.List;
-
-/**
- * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}.
- * This is the only executor that {@link org.apache.flink.table.planner.BatchPlanner} supports.
- */
-@Internal
-public class BatchExecutor extends ExecutorBase {
-
-	@VisibleForTesting
-	public BatchExecutor(StreamExecutionEnvironment executionEnvironment) {
-		super(executionEnvironment);
-	}
-
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-		StreamExecutionEnvironment execEnv = getExecutionEnvironment();
-		StreamGraph streamGraph = generateStreamGraph(transformations, jobName);
-		return execEnv.execute(streamGraph);
-	}
-
-	/**
-	 * Sets batch configs.
-	 */
-	private void setBatchProperties(StreamExecutionEnvironment execEnv) {
-		ExecutionConfig executionConfig = execEnv.getConfig();
-		executionConfig.enableObjectReuse();
-		executionConfig.setLatencyTrackingInterval(-1);
-		execEnv.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
-		execEnv.setBufferTimeout(-1);
-		if (isShuffleModeAllBatch()) {
-			executionConfig.setDefaultInputDependencyConstraint(InputDependencyConstraint.ALL);
-		}
-	}
-
-	/**
-	 * Translates transformationList to streamGraph.
-	 */
-	public StreamGraph generateStreamGraph(List<Transformation<?>> transformations, String jobName) {
-		StreamExecutionEnvironment execEnv = getExecutionEnvironment();
-		setBatchProperties(execEnv);
-		transformations.forEach(execEnv::addOperator);
-		StreamGraph streamGraph;
-		streamGraph = execEnv.getStreamGraph(getNonEmptyJobName(jobName));
-		// All transformations should set managed memory size.
-		ResourceSpec managedResourceSpec = NodeResourceUtil.fromManagedMem(0);
-		streamGraph.getStreamNodes().forEach(sn -> {
-			if (sn.getMinResources().equals(ResourceSpec.DEFAULT)) {
-				sn.setResources(managedResourceSpec, managedResourceSpec);
-			}
-		});
-		streamGraph.setChaining(true);
-		streamGraph.setScheduleMode(ScheduleMode.LAZY_FROM_SOURCES_WITH_BATCH_SLOT_REQUEST);
-		streamGraph.setStateBackend(null);
-		if (streamGraph.getCheckpointConfig().isCheckpointingEnabled()) {
-			throw new IllegalArgumentException("Checkpoint is not supported for batch jobs.");
-		}
-		if (isShuffleModeAllBatch()) {
-			streamGraph.setBlockingConnectionsBetweenChains(true);
-		}
-		return streamGraph;
-	}
-
-	private boolean isShuffleModeAllBatch() {
-		String value = tableConfig.getConfiguration().getString(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE);
-		if (value.equalsIgnoreCase(ShuffleMode.BATCH.toString())) {
-			return true;
-		} else if (!value.equalsIgnoreCase(ShuffleMode.PIPELINED.toString())) {
-			throw new IllegalArgumentException(ExecutionConfigOptions.SQL_EXEC_SHUFFLE_MODE.key() +
-					" can only be set to " + ShuffleMode.BATCH.toString() + " or " + ShuffleMode.PIPELINED.toString());
-		}
-		return false;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BlinkExecutorFactory.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BlinkExecutorFactory.java
deleted file mode 100644
index 855aa4c..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/BlinkExecutorFactory.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.executor;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.delegation.Executor;
-import org.apache.flink.table.delegation.ExecutorFactory;
-import org.apache.flink.table.descriptors.DescriptorProperties;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Factory to create an implementation of {@link Executor} to use in a
- * {@link org.apache.flink.table.api.TableEnvironment}. The {@link org.apache.flink.table.api.TableEnvironment}
- * should use {@link #create(Map)} method that does not bind to any particular environment,
- * whereas {@link org.apache.flink.table.api.scala.StreamTableEnvironment} should use
- * {@link #create(Map, StreamExecutionEnvironment)} as it is always backed by
- * some {@link StreamExecutionEnvironment}
- */
-@Internal
-public class BlinkExecutorFactory implements ExecutorFactory {
-
-	/**
-	 * Creates a corresponding {@link ExecutorBase}.
-	 *
-	 * @param properties Static properties of the {@link Executor}, the same that were used for factory lookup.
-	 * @param executionEnvironment a {@link StreamExecutionEnvironment} to use while executing Table programs.
-	 * @return instance of a {@link Executor}
-	 */
-	public Executor create(Map<String, String> properties, StreamExecutionEnvironment executionEnvironment) {
-		if (Boolean.valueOf(properties.getOrDefault(EnvironmentSettings.STREAMING_MODE, "true"))) {
-			return new StreamExecutor(executionEnvironment);
-		} else {
-			return new BatchExecutor(executionEnvironment);
-		}
-	}
-
-	@Override
-	public Executor create(Map<String, String> properties) {
-		return create(properties, StreamExecutionEnvironment.getExecutionEnvironment());
-	}
-
-	@Override
-	public Map<String, String> requiredContext() {
-		DescriptorProperties properties = new DescriptorProperties();
-		return properties.asMap();
-	}
-
-	@Override
-	public List<String> supportedProperties() {
-		return Arrays.asList(EnvironmentSettings.STREAMING_MODE, EnvironmentSettings.CLASS_NAME);
-	}
-
-	@Override
-	public Map<String, String> optionalContext() {
-		Map<String, String> context = new HashMap<>();
-		context.put(EnvironmentSettings.CLASS_NAME, this.getClass().getCanonicalName());
-		return context;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/ExecutorBase.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/ExecutorBase.java
deleted file mode 100644
index 4d67547..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/ExecutorBase.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.executor;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.table.api.TableConfig;
-import org.apache.flink.table.delegation.Executor;
-import org.apache.flink.util.StringUtils;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}.
- */
-@Internal
-public abstract class ExecutorBase implements Executor {
-
-	private static final String DEFAULT_JOB_NAME = "Flink Exec Table Job";
-
-	private final StreamExecutionEnvironment executionEnvironment;
-	protected List<Transformation<?>> transformations = new ArrayList<>();
-	protected TableConfig tableConfig;
-
-	public ExecutorBase(StreamExecutionEnvironment executionEnvironment) {
-		this.executionEnvironment = executionEnvironment;
-	}
-
-	public void setTableConfig(TableConfig tableConfig) {
-		this.tableConfig = tableConfig;
-	}
-
-	@Override
-	public void apply(List<Transformation<?>> transformations) {
-		this.transformations.addAll(transformations);
-	}
-
-	public StreamExecutionEnvironment getExecutionEnvironment() {
-		return executionEnvironment;
-	}
-
-	public abstract StreamGraph generateStreamGraph(
-			List<Transformation<?>> transformations,
-			String jobName) throws Exception;
-
-	protected String getNonEmptyJobName(String jobName) {
-		if (StringUtils.isNullOrWhitespaceOnly(jobName)) {
-			return DEFAULT_JOB_NAME;
-		} else {
-			return jobName;
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/StreamExecutor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/StreamExecutor.java
deleted file mode 100644
index 9df4ea3..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/executor/StreamExecutor.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.executor;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.table.delegation.Executor;
-
-import java.util.List;
-
-/**
- * An implementation of {@link Executor} that is backed by a {@link StreamExecutionEnvironment}.
- * This is the only executor that {@link org.apache.flink.table.planner.StreamPlanner} supports.
- */
-@Internal
-public class StreamExecutor extends ExecutorBase {
-
-	@VisibleForTesting
-	public StreamExecutor(StreamExecutionEnvironment executionEnvironment) {
-		super(executionEnvironment);
-	}
-
-	@Override
-	public JobExecutionResult execute(String jobName) throws Exception {
-		StreamExecutionEnvironment execEnv = getExecutionEnvironment();
-		return execEnv.execute(generateStreamGraph(transformations, jobName));
-	}
-
-	public StreamGraph generateStreamGraph(
-			List<Transformation<?>> transformations,
-			String jobName) throws Exception {
-		transformations.forEach(getExecutionEnvironment()::addOperator);
-		return getExecutionEnvironment().getStreamGraph(getNonEmptyJobName(jobName));
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ExpressionBuilder.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ExpressionBuilder.java
deleted file mode 100644
index 333d763..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ExpressionBuilder.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
-import org.apache.flink.table.functions.FunctionDefinition;
-import org.apache.flink.table.types.DataType;
-
-import java.util.List;
-
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AND;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.CAST;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.CONCAT;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.DIVIDE;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.EQUALS;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.GREATER_THAN;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.IF;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.IS_NULL;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.LESS_THAN;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.MINUS;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.MOD;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.NOT;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.OR;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.PLUS;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.REINTERPRET_CAST;
-import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.TIMES;
-import static org.apache.flink.table.functions.InternalFunctionDefinitions.THROW_EXCEPTION;
-
-/**
- * Builder for {@link Expression}s.
- */
-public class ExpressionBuilder {
-
-	public static Expression nullOf(DataType type) {
-		return literal(null, type);
-	}
-
-	public static Expression literal(Object value) {
-		return ApiExpressionUtils.valueLiteral(value);
-	}
-
-	public static Expression literal(Object value, DataType type) {
-		return ApiExpressionUtils.valueLiteral(value, type);
-	}
-
-	public static Expression call(FunctionDefinition functionDefinition, Expression... args) {
-		return ApiExpressionUtils.unresolvedCall(functionDefinition, args);
-	}
-
-	public static Expression call(FunctionDefinition functionDefinition, List<Expression> args) {
-		return ApiExpressionUtils.unresolvedCall(functionDefinition, args.toArray(new Expression[0]));
-	}
-
-	public static Expression and(Expression arg1, Expression arg2) {
-		return call(AND, arg1, arg2);
-	}
-
-	public static Expression or(Expression arg1, Expression arg2) {
-		return call(OR, arg1, arg2);
-	}
-
-	public static Expression not(Expression arg) {
-		return call(NOT, arg);
-	}
-
-	public static Expression isNull(Expression input) {
-		return call(IS_NULL, input);
-	}
-
-	public static Expression ifThenElse(Expression condition, Expression ifTrue,
-			Expression ifFalse) {
-		return call(IF, condition, ifTrue, ifFalse);
-	}
-
-	public static Expression plus(Expression input1, Expression input2) {
-		return call(PLUS, input1, input2);
-	}
-
-	public static Expression minus(Expression input1, Expression input2) {
-		return call(MINUS, input1, input2);
-	}
-
-	public static Expression div(Expression input1, Expression input2) {
-		return call(DIVIDE, input1, input2);
-	}
-
-	public static Expression times(Expression input1, Expression input2) {
-		return call(TIMES, input1, input2);
-	}
-
-	public static Expression mod(Expression input1, Expression input2) {
-		return call(MOD, input1, input2);
-	}
-
-	public static Expression equalTo(Expression input1, Expression input2) {
-		return call(EQUALS, input1, input2);
-	}
-
-	public static Expression lessThan(Expression input1, Expression input2) {
-		return call(LESS_THAN, input1, input2);
-	}
-
-	public static Expression greaterThan(Expression input1, Expression input2) {
-		return call(GREATER_THAN, input1, input2);
-	}
-
-	public static Expression cast(Expression child, Expression type) {
-		return call(CAST, child, type);
-	}
-
-	public static Expression reinterpretCast(Expression child, Expression type,
-			boolean checkOverflow) {
-		return call(REINTERPRET_CAST, child, type, literal(checkOverflow));
-	}
-
-	public static TypeLiteralExpression typeLiteral(DataType type) {
-		return ApiExpressionUtils.typeLiteral(type);
-	}
-
-	public static Expression concat(Expression input1, Expression input2) {
-		return call(CONCAT, input1, input2);
-	}
-
-	public static Expression throwException(String msg, DataType type) {
-		return call(THROW_EXCEPTION, typeLiteral(type));
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
deleted file mode 100644
index 72ddfa3..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/PlannerTypeInferenceUtilImpl.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.delegation.PlannerTypeInferenceUtil;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.inference.TypeInferenceUtil;
-import org.apache.flink.table.typeutils.TypeCoercion;
-import org.apache.flink.table.validate.ValidationFailure;
-import org.apache.flink.table.validate.ValidationResult;
-
-import java.util.List;
-import java.util.Optional;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
-import static org.apache.flink.table.util.JavaScalaConversionUtil.toJava;
-
-/**
- * Implementation of {@link PlannerTypeInferenceUtil}.
- */
-@Internal
-public final class PlannerTypeInferenceUtilImpl implements PlannerTypeInferenceUtil {
-
-	public static final PlannerTypeInferenceUtil INSTANCE = new PlannerTypeInferenceUtilImpl();
-
-	private static final PlannerExpressionConverter CONVERTER = PlannerExpressionConverter.INSTANCE();
-
-	@Override
-	public TypeInferenceUtil.Result runTypeInference(
-			UnresolvedCallExpression unresolvedCall,
-			List<ResolvedExpression> resolvedArgs) {
-		final PlannerExpression plannerCall = unresolvedCall.accept(CONVERTER);
-
-		if (plannerCall instanceof InputTypeSpec) {
-			return resolveWithCastedAssignment(
-				unresolvedCall,
-				resolvedArgs,
-				toJava(((InputTypeSpec) plannerCall).expectedTypes()),
-				plannerCall.resultType());
-		} else {
-			validateArguments(plannerCall);
-
-			final List<DataType> expectedArgumentTypes = resolvedArgs.stream()
-				.map(ResolvedExpression::getOutputDataType)
-				.collect(Collectors.toList());
-
-			return new TypeInferenceUtil.Result(
-				expectedArgumentTypes,
-				null,
-				fromLegacyInfoToDataType(plannerCall.resultType()));
-		}
-	}
-
-	private TypeInferenceUtil.Result resolveWithCastedAssignment(
-			UnresolvedCallExpression unresolvedCall,
-			List<ResolvedExpression> args,
-			List<TypeInformation<?>> expectedTypes,
-			TypeInformation<?> resultType) {
-
-		final List<PlannerExpression> plannerArgs = unresolvedCall.getChildren()
-			.stream()
-			.map(e -> e.accept(CONVERTER))
-			.collect(Collectors.toList());
-
-		final List<DataType> castedArgs = IntStream.range(0, plannerArgs.size())
-			.mapToObj(idx -> castIfNeeded(
-				args.get(idx),
-				plannerArgs.get(idx),
-				expectedTypes.get(idx)))
-			.collect(Collectors.toList());
-
-		return new TypeInferenceUtil.Result(
-			castedArgs,
-			null,
-			fromLegacyInfoToDataType(resultType));
-	}
-
-	private void validateArguments(PlannerExpression plannerCall) {
-		if (!plannerCall.valid()) {
-			throw new ValidationException(
-				getValidationErrorMessage(plannerCall)
-					.orElse("Unexpected behavior, validation failed but can't get error messages!"));
-		}
-	}
-
-	/**
-	 * Return the validation error message of this {@link PlannerExpression} or return the
-	 * validation error message of it's children if it passes the validation. Return empty if
-	 * all validation succeeded.
-	 */
-	private Optional<String> getValidationErrorMessage(PlannerExpression plannerCall) {
-		ValidationResult validationResult = plannerCall.validateInput();
-		if (validationResult instanceof ValidationFailure) {
-			return Optional.of(((ValidationFailure) validationResult).message());
-		} else {
-			for (Expression plannerExpression: plannerCall.getChildren()) {
-				Optional<String> errorMessage = getValidationErrorMessage((PlannerExpression) plannerExpression);
-				if (errorMessage.isPresent()) {
-					return errorMessage;
-				}
-			}
-		}
-		return Optional.empty();
-	}
-
-	private DataType castIfNeeded(
-			ResolvedExpression child,
-			PlannerExpression plannerChild,
-			TypeInformation<?> expectedType) {
-		TypeInformation<?> actualType = plannerChild.resultType();
-		if (actualType.equals(expectedType)) {
-			return child.getOutputDataType();
-		} else if (TypeCoercion.canSafelyCast(
-				fromTypeInfoToLogicalType(actualType), fromTypeInfoToLogicalType(expectedType))) {
-			return fromLegacyInfoToDataType(expectedType);
-		} else {
-			throw new ValidationException(String.format("Incompatible type of argument: %s Expected: %s",
-				child,
-				expectedType));
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggInputReference.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggInputReference.java
deleted file mode 100644
index 19e2b35..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggInputReference.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Normally we should use {@link FieldReferenceExpression} to represent an input field.
- * {@link FieldReferenceExpression} uses name to locate the field, in aggregate case, we want to use
- * field index.
- */
-public class ResolvedAggInputReference implements Expression {
-
-	private final String name;
-	private final int index;
-	private final LogicalType resultType;
-
-	public ResolvedAggInputReference(String name, int index, LogicalType resultType) {
-		this.name = Preconditions.checkNotNull(name);
-		this.index = index;
-		this.resultType = resultType;
-	}
-
-	public String getName() {
-		return name;
-	}
-
-	public int getIndex() {
-		return index;
-	}
-
-	public LogicalType getResultType() {
-		return resultType;
-	}
-
-	@Override
-	public String asSummaryString() {
-		return name;
-	}
-
-	@Override
-	public List<Expression> getChildren() {
-		return Collections.emptyList();
-	}
-
-	@Override
-	public <R> R accept(ExpressionVisitor<R> visitor) {
-		return visitor.visit(this);
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		ResolvedAggInputReference that = (ResolvedAggInputReference) o;
-		return index == that.index && name.equals(that.name) && resultType.equals(that.resultType);
-	}
-
-	@Override
-	public int hashCode() {
-		int result = name.hashCode();
-		result = 31 * result + index;
-		result = 31 * result + resultType.hashCode();
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return asSummaryString();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggLocalReference.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggLocalReference.java
deleted file mode 100644
index 5884c42..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedAggLocalReference.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.types.logical.LogicalType;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Special reference which represent a local filed, such as aggregate buffers or constants.
- * We are stored as class members, so the field can be referenced directly.
- * We should use an unique name to locate the field.
- *
- * <p>See {@link org.apache.flink.table.codegen.ExprCodeGenerator#visitLocalRef}.
- */
-public class ResolvedAggLocalReference implements Expression {
-
-	private final String fieldTerm;
-	private final String nullTerm;
-	private final LogicalType resultType;
-
-	public ResolvedAggLocalReference(String fieldTerm, String nullTerm, LogicalType resultType) {
-		this.fieldTerm = fieldTerm;
-		this.nullTerm = nullTerm;
-		this.resultType = resultType;
-	}
-
-	public String getFieldTerm() {
-		return fieldTerm;
-	}
-
-	public String getNullTerm() {
-		return nullTerm;
-	}
-
-	public LogicalType getResultType() {
-		return resultType;
-	}
-
-	@Override
-	public String asSummaryString() {
-		return fieldTerm;
-	}
-
-	@Override
-	public List<Expression> getChildren() {
-		return Collections.emptyList();
-	}
-
-	@Override
-	public <R> R accept(ExpressionVisitor<R> visitor) {
-		return visitor.visit(this);
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		ResolvedAggLocalReference that = (ResolvedAggLocalReference) o;
-
-		return fieldTerm.equals(that.fieldTerm) && nullTerm.equals(that.nullTerm) && resultType.equals(that.resultType);
-	}
-
-	@Override
-	public int hashCode() {
-		int result = fieldTerm.hashCode();
-		result = 31 * result + nullTerm.hashCode();
-		result = 31 * result + resultType.hashCode();
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return asSummaryString();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedDistinctKeyReference.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedDistinctKeyReference.java
deleted file mode 100644
index 34339ca..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/ResolvedDistinctKeyReference.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Resolved distinct key reference.
- */
-public class ResolvedDistinctKeyReference implements Expression {
-
-	private final String name;
-	private final LogicalType resultType;
-
-	public ResolvedDistinctKeyReference(String name, LogicalType resultType) {
-		this.name = Preconditions.checkNotNull(name);
-		this.resultType = resultType;
-	}
-
-	public String getName() {
-		return name;
-	}
-
-	public LogicalType getResultType() {
-		return resultType;
-	}
-
-	@Override
-	public String asSummaryString() {
-		return name;
-	}
-
-	@Override
-	public List<Expression> getChildren() {
-		return Collections.emptyList();
-	}
-
-	@Override
-	public <R> R accept(ExpressionVisitor<R> visitor) {
-		return visitor.visit(this);
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		ResolvedDistinctKeyReference that = (ResolvedDistinctKeyReference) o;
-
-		return name.equals(that.name) && resultType.equals(that.resultType);
-	}
-
-	@Override
-	public int hashCode() {
-		int result = name.hashCode();
-		result = 31 * result + resultType.hashCode();
-		return result;
-	}
-
-	@Override
-	public String toString() {
-		return asSummaryString();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
deleted file mode 100644
index 5de269f..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeConverter.java
+++ /dev/null
@@ -1,1015 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.calcite.FlinkContext;
-import org.apache.flink.table.calcite.FlinkPlannerImpl;
-import org.apache.flink.table.calcite.FlinkRelBuilder;
-import org.apache.flink.table.calcite.FlinkTypeFactory;
-import org.apache.flink.table.calcite.RexAggLocalVariable;
-import org.apache.flink.table.calcite.RexDistinctKeyVariable;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.functions.AggregateFunctionDefinition;
-import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
-import org.apache.flink.table.functions.FunctionDefinition;
-import org.apache.flink.table.functions.ScalarFunction;
-import org.apache.flink.table.functions.ScalarFunctionDefinition;
-import org.apache.flink.table.functions.TableFunctionDefinition;
-import org.apache.flink.table.functions.UserDefinedAggregateFunction;
-import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable;
-import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils;
-import org.apache.flink.table.operations.QueryOperation;
-import org.apache.flink.table.types.logical.ArrayType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeRoot;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
-import org.apache.flink.util.Preconditions;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.avatica.util.DateTimeUtils;
-import org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexFieldCollation;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexSubQuery;
-import org.apache.calcite.rex.RexWindowBound;
-import org.apache.calcite.sql.SqlAggFunction;
-import org.apache.calcite.sql.SqlBasicCall;
-import org.apache.calcite.sql.SqlFunction;
-import org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlPostfixOperator;
-import org.apache.calcite.sql.SqlWindow;
-import org.apache.calcite.sql.fun.SqlTrimFunction;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.OrdinalReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.util.DateString;
-import org.apache.calcite.util.TimeString;
-import org.apache.calcite.util.TimestampString;
-import org.apache.calcite.util.TimestampWithTimeZoneString;
-
-import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.stream.Collectors;
-
-import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
-import static org.apache.flink.table.calcite.FlinkTypeFactory.toLogicalType;
-import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
-import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isCharacterString;
-import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTemporal;
-import static org.apache.flink.table.runtime.typeutils.TypeCheckUtils.isTimeInterval;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
-
-/**
- * Visit expression to generator {@link RexNode}.
- *
- * <p>TODO actually we should use {@link ResolvedExpressionVisitor} here as it is the output of the API.
- * we will update it after introduce Expression resolve in AggCodeGen.
- */
-public class RexNodeConverter implements ExpressionVisitor<RexNode> {
-
-	private final RelBuilder relBuilder;
-	private final FlinkTypeFactory typeFactory;
-
-	// store mapping from BuiltInFunctionDefinition to it's RexNodeConversion.
-	private final Map<FunctionDefinition, RexNodeConversion> conversionsOfBuiltInFunc = new IdentityHashMap<>();
-
-	public RexNodeConverter(RelBuilder relBuilder) {
-		this.relBuilder = relBuilder;
-		this.typeFactory = (FlinkTypeFactory) relBuilder.getRexBuilder().getTypeFactory();
-
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.CAST, exprs -> convertCast(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.REINTERPRET_CAST, exprs -> convertReinterpretCast(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.IN, exprs -> convertIn(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.GET, exprs -> convertGet(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.TRIM, exprs -> convertTrim(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.AS, exprs -> convertAs(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.OVER, exprs -> convertOver(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.IS_NULL, exprs -> convertIsNull(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.BETWEEN, exprs -> convertBetween(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.NOT_BETWEEN, exprs -> convertNotBetween(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.REPLACE, exprs -> convertReplace(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.PLUS, exprs -> convertPlus(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.CEIL, exprs -> convertCeil(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.FLOOR, exprs -> convertFloor(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.TEMPORAL_OVERLAPS, exprs -> convertTemporalOverlaps(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.TIMESTAMP_DIFF, exprs -> convertTimestampDiff(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ARRAY, exprs -> convertArray(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ARRAY_ELEMENT, exprs -> convertArrayElement(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.MAP, exprs -> convertMap(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ROW, exprs -> convertRow(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ORDER_ASC, exprs -> convertOrderAsc(exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SHA1, exprs -> convert(FlinkSqlOperatorTable.SHA1, exprs));
-		// logic functions
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.AND, exprs -> convert(FlinkSqlOperatorTable.AND, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.OR, exprs -> convert(FlinkSqlOperatorTable.OR, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.NOT, exprs -> convert(FlinkSqlOperatorTable.NOT, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.IF, exprs -> convert(FlinkSqlOperatorTable.CASE, exprs));
-
-		// comparison functions
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.EQUALS, exprs -> convert(FlinkSqlOperatorTable.EQUALS, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.GREATER_THAN, exprs -> convert(FlinkSqlOperatorTable.GREATER_THAN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, exprs -> convert(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.LESS_THAN, exprs -> convert(FlinkSqlOperatorTable.LESS_THAN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, exprs -> convert(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.NOT_EQUALS, exprs -> convert(FlinkSqlOperatorTable.NOT_EQUALS, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_NULL, exprs -> convert(FlinkSqlOperatorTable.IS_NULL, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_NOT_NULL, exprs -> convert(FlinkSqlOperatorTable.IS_NOT_NULL, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_TRUE, exprs -> convert(FlinkSqlOperatorTable.IS_TRUE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_FALSE, exprs -> convert(FlinkSqlOperatorTable.IS_FALSE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_NOT_TRUE, exprs -> convert(FlinkSqlOperatorTable.IS_NOT_TRUE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.IS_NOT_FALSE, exprs -> convert(FlinkSqlOperatorTable.IS_NOT_FALSE, exprs));
-
-		// string functions
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CHAR_LENGTH, exprs -> convert(FlinkSqlOperatorTable.CHAR_LENGTH, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.INIT_CAP, exprs -> convert(FlinkSqlOperatorTable.INITCAP, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LIKE, exprs -> convert(FlinkSqlOperatorTable.LIKE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.LOWER, exprs -> convert(FlinkSqlOperatorTable.LOWER, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SIMILAR, exprs -> convert(FlinkSqlOperatorTable.SIMILAR_TO, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SUBSTRING, exprs -> convert(FlinkSqlOperatorTable.SUBSTRING, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.UPPER, exprs -> convert(FlinkSqlOperatorTable.UPPER, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.POSITION, exprs -> convert(FlinkSqlOperatorTable.POSITION, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.OVERLAY, exprs -> convert(FlinkSqlOperatorTable.OVERLAY, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CONCAT, exprs -> convert(FlinkSqlOperatorTable.CONCAT_FUNCTION, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CONCAT_WS, exprs -> convert(FlinkSqlOperatorTable.CONCAT_WS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LPAD, exprs -> convert(FlinkSqlOperatorTable.LPAD, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.RPAD, exprs -> convert(FlinkSqlOperatorTable.RPAD, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.REGEXP_EXTRACT, exprs -> convert(FlinkSqlOperatorTable.REGEXP_EXTRACT, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.FROM_BASE64, exprs -> convert(FlinkSqlOperatorTable.FROM_BASE64, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.TO_BASE64, exprs -> convert(FlinkSqlOperatorTable.TO_BASE64, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.UUID, exprs -> convert(FlinkSqlOperatorTable.UUID, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.LTRIM, exprs -> convert(FlinkSqlOperatorTable.LTRIM, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.RTRIM, exprs -> convert(FlinkSqlOperatorTable.RTRIM, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.REPEAT, exprs -> convert(FlinkSqlOperatorTable.REPEAT, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.REGEXP_REPLACE, exprs -> convert(FlinkSqlOperatorTable.REGEXP_REPLACE, exprs));
-
-		// math functions
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.MINUS, exprs -> convert(FlinkSqlOperatorTable.MINUS, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.DIVIDE, exprs -> convert(FlinkSqlOperatorTable.DIVIDE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.TIMES, exprs -> convert(FlinkSqlOperatorTable.MULTIPLY, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ABS, exprs -> convert(FlinkSqlOperatorTable.ABS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.EXP, exprs -> convert(FlinkSqlOperatorTable.EXP, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.LOG10, exprs -> convert(FlinkSqlOperatorTable.LOG10, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LOG2, exprs -> convert(FlinkSqlOperatorTable.LOG2, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LN, exprs -> convert(FlinkSqlOperatorTable.LN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LOG, exprs -> convert(FlinkSqlOperatorTable.LOG, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.POWER, exprs -> convert(FlinkSqlOperatorTable.POWER, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.MOD, exprs -> convert(FlinkSqlOperatorTable.MOD, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SQRT, exprs -> convert(FlinkSqlOperatorTable.SQRT, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.MINUS_PREFIX, exprs -> convert(FlinkSqlOperatorTable.UNARY_MINUS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SIN, exprs -> convert(FlinkSqlOperatorTable.SIN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.COS, exprs -> convert(FlinkSqlOperatorTable.COS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SINH, exprs -> convert(FlinkSqlOperatorTable.SINH, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.TAN, exprs -> convert(FlinkSqlOperatorTable.TAN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.TANH, exprs -> convert(FlinkSqlOperatorTable.TANH, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.COT, exprs -> convert(FlinkSqlOperatorTable.COT, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ASIN, exprs -> convert(FlinkSqlOperatorTable.ASIN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ACOS, exprs -> convert(FlinkSqlOperatorTable.ACOS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.ATAN, exprs -> convert(FlinkSqlOperatorTable.ATAN, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.ATAN2, exprs -> convert(FlinkSqlOperatorTable.ATAN2, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.COSH, exprs -> convert(FlinkSqlOperatorTable.COSH, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.DEGREES, exprs -> convert(FlinkSqlOperatorTable.DEGREES, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.RADIANS, exprs -> convert(FlinkSqlOperatorTable.RADIANS, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SIGN, exprs -> convert(FlinkSqlOperatorTable.SIGN, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.ROUND, exprs -> convert(FlinkSqlOperatorTable.ROUND, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.PI, exprs -> convert(FlinkSqlOperatorTable.PI, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.E, exprs -> convert(FlinkSqlOperatorTable.E, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.RAND, exprs -> convert(FlinkSqlOperatorTable.RAND, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.RAND_INTEGER, exprs -> convert(FlinkSqlOperatorTable.RAND_INTEGER, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.BIN, exprs -> convert(FlinkSqlOperatorTable.BIN, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.HEX, exprs -> convert(FlinkSqlOperatorTable.HEX, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.TRUNCATE, exprs -> convert(FlinkSqlOperatorTable.TRUNCATE, exprs));
-
-		// time functions
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.EXTRACT, exprs -> convert(FlinkSqlOperatorTable.EXTRACT, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CURRENT_DATE, exprs -> convert(FlinkSqlOperatorTable.CURRENT_DATE, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CURRENT_TIME, exprs -> convert(FlinkSqlOperatorTable.CURRENT_TIME, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.CURRENT_TIMESTAMP, exprs -> convert(FlinkSqlOperatorTable.CURRENT_TIMESTAMP, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.LOCAL_TIME, exprs -> convert(FlinkSqlOperatorTable.LOCALTIME, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.LOCAL_TIMESTAMP, exprs -> convert(FlinkSqlOperatorTable.LOCALTIMESTAMP, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.DATE_FORMAT, exprs -> convert(FlinkSqlOperatorTable.DATE_FORMAT, exprs));
-
-		// collection
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.AT, exprs -> convert(FlinkSqlOperatorTable.ITEM, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.CARDINALITY, exprs -> convert(FlinkSqlOperatorTable.CARDINALITY, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.ORDER_DESC, exprs -> convert(FlinkSqlOperatorTable.DESC, exprs));
-
-		// crypto hash
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.MD5, exprs -> convert(FlinkSqlOperatorTable.MD5, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SHA2, exprs -> convert(FlinkSqlOperatorTable.SHA2, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SHA224, exprs -> convert(FlinkSqlOperatorTable.SHA224, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SHA256, exprs -> convert(FlinkSqlOperatorTable.SHA256, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SHA384, exprs -> convert(FlinkSqlOperatorTable.SHA384, exprs));
-		conversionsOfBuiltInFunc
-				.put(BuiltInFunctionDefinitions.SHA512, exprs -> convert(FlinkSqlOperatorTable.SHA512, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SHA1, exprs -> convert(FlinkSqlOperatorTable.SHA1, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.STREAM_RECORD_TIMESTAMP,
-				exprs -> convert(FlinkSqlOperatorTable.STREAMRECORD_TIMESTAMP, exprs));
-	}
-
-	@Override
-	public RexNode visit(CallExpression call) {
-		FunctionDefinition func = call.getFunctionDefinition();
-		if (func instanceof ScalarFunctionDefinition) {
-			ScalarFunction scalaFunc = ((ScalarFunctionDefinition) func).getScalarFunction();
-			List<RexNode> child = convertCallChildren(call.getChildren());
-			SqlFunction sqlFunction = UserDefinedFunctionUtils.createScalarSqlFunction(
-					scalaFunc.functionIdentifier(),
-					scalaFunc.toString(),
-					scalaFunc,
-					typeFactory);
-			return relBuilder.call(sqlFunction, child);
-		} else if (func instanceof TableFunctionDefinition) {
-			throw new RuntimeException("There is no possible reach here!");
-		} else if (func instanceof AggregateFunctionDefinition) {
-			UserDefinedAggregateFunction aggFunc = ((AggregateFunctionDefinition) func).getAggregateFunction();
-			if (aggFunc instanceof AggregateFunction) {
-				SqlFunction aggSqlFunction = UserDefinedFunctionUtils.createAggregateSqlFunction(
-						aggFunc.functionIdentifier(),
-						aggFunc.toString(),
-						(AggregateFunction) aggFunc,
-						fromLegacyInfoToDataType(aggFunc.getResultType()),
-						fromLegacyInfoToDataType(aggFunc.getAccumulatorType()),
-						typeFactory);
-				List<RexNode> child = convertCallChildren(call.getChildren());
-				return relBuilder.call(aggSqlFunction, child);
-			} else {
-				throw new UnsupportedOperationException("TableAggregateFunction is not supported yet!");
-			}
-
-		} else {
-			FunctionDefinition def = call.getFunctionDefinition();
-			if (conversionsOfBuiltInFunc.containsKey(def)) {
-				RexNodeConversion conversion = conversionsOfBuiltInFunc.get(def);
-				return conversion.convert(call);
-			} else {
-				throw new UnsupportedOperationException(def.toString());
-			}
-		}
-	}
-
-	private List<RexNode> convertCallChildren(List<Expression> children) {
-		return children.stream()
-				.map(expression -> expression.accept(RexNodeConverter.this))
-				.collect(Collectors.toList());
-	}
-
-	private RexNode convert(SqlOperator sqlOperator, List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		return relBuilder.call(sqlOperator, childrenRexNode);
-	}
-
-	private RexNode convertArrayElement(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		return relBuilder.call(FlinkSqlOperatorTable.ELEMENT, childrenRexNode);
-	}
-
-	private RexNode convertOrderAsc(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		return childrenRexNode.get(0);
-	}
-
-	private RexNode convertTimestampDiff(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		return relBuilder.call(FlinkSqlOperatorTable.TIMESTAMP_DIFF, childrenRexNode.get(0), childrenRexNode.get(2),
-				childrenRexNode.get(1));
-	}
-
-	private RexNode convertIsNull(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		return relBuilder.isNull(childrenRexNode.get(0));
-	}
-
-	private RexNode convertNotBetween(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		Preconditions.checkArgument(childrenRexNode.size() == 3);
-		RexNode expr = childrenRexNode.get(0);
-		RexNode lowerBound = childrenRexNode.get(1);
-		RexNode upperBound = childrenRexNode.get(2);
-		return relBuilder.or(
-				relBuilder.call(FlinkSqlOperatorTable.LESS_THAN, expr, lowerBound),
-				relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN, expr, upperBound));
-	}
-
-	private RexNode convertBetween(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		Preconditions.checkArgument(childrenRexNode.size() == 3);
-		RexNode expr = childrenRexNode.get(0);
-		RexNode lowerBound = childrenRexNode.get(1);
-		RexNode upperBound = childrenRexNode.get(2);
-		return relBuilder.and(
-				relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, expr, lowerBound),
-				relBuilder.call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, expr, upperBound));
-	}
-
-	private RexNode convertCeil(List<Expression> children) {
-		Preconditions.checkArgument(children.size() == 1 || children.size() == 2);
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		if (children.size() == 1) {
-			return relBuilder.call(FlinkSqlOperatorTable.CEIL, childrenRexNode);
-		} else {
-			return relBuilder.call(FlinkSqlOperatorTable.CEIL, childrenRexNode.get(1), childrenRexNode.get(0));
-		}
-	}
-
-	private RexNode convertFloor(List<Expression> children) {
-		Preconditions.checkArgument(children.size() == 1 || children.size() == 2);
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		if (children.size() == 1) {
-			return relBuilder.call(FlinkSqlOperatorTable.FLOOR, childrenRexNode);
-		} else {
-			return relBuilder.call(FlinkSqlOperatorTable.FLOOR, childrenRexNode.get(1), childrenRexNode.get(0));
-		}
-	}
-
-	private RexNode convertArray(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		ArrayType arrayType = new ArrayType(toLogicalType(childrenRexNode.get(0).getType()));
-		// TODO get type from CallExpression directly
-		RelDataType relDataType = typeFactory.createFieldTypeFromLogicalType(arrayType);
-		return relBuilder.getRexBuilder().makeCall(relDataType, FlinkSqlOperatorTable.ARRAY_VALUE_CONSTRUCTOR, childrenRexNode);
-	}
-
-	private RexNode convertMap(List<Expression> children) {
-		Preconditions.checkArgument(!children.isEmpty() && children.size() % 2 == 0);
-		// TODO get type from CallExpression directly
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		RelDataType keyType = childrenRexNode.get(0).getType();
-		RelDataType valueType = childrenRexNode.get(childrenRexNode.size() - 1).getType();
-		RelDataType mapType = typeFactory.createMapType(keyType, valueType);
-		return relBuilder.getRexBuilder().makeCall(mapType, FlinkSqlOperatorTable.MAP_VALUE_CONSTRUCTOR, childrenRexNode);
-	}
-
-	private RexNode convertRow(List<Expression> children) {
-		// TODO get type from CallExpression directly
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		LogicalType[] childTypes = childrenRexNode.stream().map(rexNode -> toLogicalType(rexNode.getType()))
-				.toArray(LogicalType[]::new);
-		RowType rowType = RowType.of(childTypes);
-		RelDataType relDataType = typeFactory.createFieldTypeFromLogicalType(rowType);
-		return relBuilder.getRexBuilder().makeCall(relDataType, FlinkSqlOperatorTable.ROW, childrenRexNode);
-	}
-
-	private RexNode convertTemporalOverlaps(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		// Standard conversion of the OVERLAPS operator.
-		// Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertOverlaps()]]
-		RexNode leftTimePoint = childrenRexNode.get(0);
-		RexNode leftTemporal = childrenRexNode.get(1);
-		RexNode rightTimePoint = childrenRexNode.get(2);
-		RexNode rightTemporal = childrenRexNode.get(3);
-		RexNode convLeftT;
-		if (isTimeInterval(toLogicalType(leftTemporal.getType()))) {
-			convLeftT = relBuilder.call(FlinkSqlOperatorTable.DATETIME_PLUS, leftTimePoint, leftTemporal);
-		} else {
-			convLeftT = leftTemporal;
-		}
-		// sort end points into start and end, such that (s0 <= e0) and (s1 <= e1).
-		RexNode leftLe = relBuilder.call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, leftTimePoint, convLeftT);
-		RexNode s0 = relBuilder.call(FlinkSqlOperatorTable.CASE, leftLe, leftTimePoint, convLeftT);
-		RexNode e0 = relBuilder.call(FlinkSqlOperatorTable.CASE, leftLe, convLeftT, leftTimePoint);
-		RexNode convRightT;
-		if (isTimeInterval(toLogicalType(rightTemporal.getType()))) {
-			convRightT = relBuilder.call(FlinkSqlOperatorTable.DATETIME_PLUS, rightTimePoint, rightTemporal);
-		} else {
-			convRightT = rightTemporal;
-		}
-		RexNode rightLe = relBuilder.call(FlinkSqlOperatorTable.LESS_THAN_OR_EQUAL, rightTimePoint, convRightT);
-		RexNode s1 = relBuilder.call(FlinkSqlOperatorTable.CASE, rightLe, rightTimePoint, convRightT);
-		RexNode e1 = relBuilder.call(FlinkSqlOperatorTable.CASE, rightLe, convRightT, rightTimePoint);
-
-		// (e0 >= s1) AND (e1 >= s0)
-		RexNode leftPred = relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, e0, s1);
-		RexNode rightPred = relBuilder.call(FlinkSqlOperatorTable.GREATER_THAN_OR_EQUAL, e1, s0);
-		return relBuilder.call(FlinkSqlOperatorTable.AND, leftPred, rightPred);
-	}
-
-	private RexNode convertPlus(List<Expression> children) {
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		if (isCharacterString(toLogicalType(childrenRexNode.get(0).getType()))) {
-			return relBuilder.call(
-					FlinkSqlOperatorTable.CONCAT,
-					childrenRexNode.get(0),
-					relBuilder.cast(childrenRexNode.get(1), VARCHAR));
-		} else if (isCharacterString(toLogicalType(childrenRexNode.get(1).getType()))) {
-			return relBuilder.call(
-					FlinkSqlOperatorTable.CONCAT,
-					relBuilder.cast(childrenRexNode.get(0), VARCHAR),
-					childrenRexNode.get(1));
-		} else if (isTimeInterval(toLogicalType(childrenRexNode.get(0).getType())) &&
-				childrenRexNode.get(0).getType() == childrenRexNode.get(1).getType()) {
-			return relBuilder.call(FlinkSqlOperatorTable.PLUS, childrenRexNode);
-		} else if (isTimeInterval(toLogicalType(childrenRexNode.get(0).getType()))
-				&& isTemporal(toLogicalType(childrenRexNode.get(1).getType()))) {
-			// Calcite has a bug that can't apply INTERVAL + DATETIME (INTERVAL at left)
-			// we manually switch them here
-			return relBuilder.call(FlinkSqlOperatorTable.DATETIME_PLUS, childrenRexNode.get(1), childrenRexNode.get(0));
-		} else if (isTemporal(toLogicalType(childrenRexNode.get(0).getType())) &&
-				isTemporal(toLogicalType(childrenRexNode.get(1).getType()))) {
-			return relBuilder.call(FlinkSqlOperatorTable.DATETIME_PLUS, childrenRexNode);
-		} else {
-			return relBuilder.call(FlinkSqlOperatorTable.PLUS, childrenRexNode);
-		}
-	}
-
-	private RexNode convertReplace(List<Expression> children) {
-		Preconditions.checkArgument(children.size() == 2 || children.size() == 3);
-		List<RexNode> childrenRexNode = convertCallChildren(children);
-		if (children.size() == 2) {
-			return relBuilder.call(
-					FlinkSqlOperatorTable.REPLACE,
-					childrenRexNode.get(0),
-					childrenRexNode.get(1),
-					relBuilder.call(FlinkSqlOperatorTable.CHAR_LENGTH, childrenRexNode.get(0)));
-		} else {
-			return relBuilder.call(FlinkSqlOperatorTable.REPLACE, childrenRexNode);
-		}
-	}
-
-	private RexNode convertOver(List<Expression> children) {
-		List<Expression> args = children;
-		Expression agg = args.get(0);
-		SqlAggFunction aggFunc = agg.accept(new SqlAggFunctionVisitor(typeFactory));
-		RelDataType aggResultType = typeFactory.createFieldTypeFromLogicalType(
-				fromDataTypeToLogicalType(((ResolvedExpression) agg).getOutputDataType()));
-
-		// assemble exprs by agg children
-		List<RexNode> aggExprs = agg.getChildren().stream().map(expr -> expr.accept(this))
-				.collect(Collectors.toList());
-
-		// assemble order by key
-		Expression orderKeyExpr = args.get(1);
-		Set<SqlKind> kinds = new HashSet<>();
-		RexNode collationRexNode = createCollation(orderKeyExpr.accept(this), RelFieldCollation.Direction.ASCENDING,
-				null, kinds);
-		ImmutableList<RexFieldCollation> orderKey = ImmutableList
-				.of(new RexFieldCollation(collationRexNode, kinds));
-
-		// assemble partition by keys
-		List<RexNode> partitionKeys = args.subList(4, args.size()).stream().map(expr -> expr.accept(this))
-				.collect(Collectors.toList());
-		// assemble bounds
-		Expression preceding = args.get(2);
-		boolean isPhysical = LogicalTypeChecks.hasRoot(
-				fromDataTypeToLogicalType(((ResolvedExpression) preceding).getOutputDataType()),
-				LogicalTypeRoot.BIGINT);
-		Expression following = args.get(3);
-		RexWindowBound lowerBound = createBound(preceding, SqlKind.PRECEDING);
-		RexWindowBound upperBound = createBound(following, SqlKind.FOLLOWING);
-
-		// build RexOver
-		return relBuilder.getRexBuilder().makeOver(
-				aggResultType,
-				aggFunc,
-				aggExprs,
-				partitionKeys,
-				orderKey,
-				lowerBound,
-				upperBound,
-				isPhysical,
-				true,
-				false,
-				false);
-	}
-
-	private RexNode convertAs(List<Expression> children) {
-		String name = extractValue((ValueLiteralExpression) children.get(1), String.class);
-		RexNode child = children.get(0).accept(this);
-		return relBuilder.alias(child, name);
-	}
-
-	private RexNode convertTrim(List<Expression> children) {
-		ValueLiteralExpression removeLeadingExpr = (ValueLiteralExpression) children.get(0);
-		Boolean removeLeading = extractValue(removeLeadingExpr, Boolean.class);
-		ValueLiteralExpression removeTrailingExpr = (ValueLiteralExpression) children.get(1);
-		Boolean removeTrailing = extractValue(removeTrailingExpr, Boolean.class);
-		RexNode trimString = children.get(2).accept(this);
-		RexNode str = children.get(3).accept(this);
-		Enum trimMode;
-		if (removeLeading && removeTrailing) {
-			trimMode = SqlTrimFunction.Flag.BOTH;
-		} else if (removeLeading) {
-			trimMode = SqlTrimFunction.Flag.LEADING;
-		} else if (removeTrailing) {
-			trimMode = SqlTrimFunction.Flag.TRAILING;
-		} else {
-			throw new IllegalArgumentException("Unsupported trim mode.");
-		}
-		return relBuilder.call(
-				FlinkSqlOperatorTable.TRIM,
-				relBuilder.getRexBuilder().makeFlag(trimMode),
-				trimString,
-				str);
-	}
-
-	private RexNode convertGet(List<Expression> children) {
-		RexNode child = children.get(0).accept(this);
-		ValueLiteralExpression keyLiteral = (ValueLiteralExpression) children.get(1);
-		Optional<Integer> indexOptional = ExpressionUtils.extractValue(keyLiteral, String.class).map(
-				child.getType().getFieldNames()::indexOf);
-		// Note: never replace the following code with :
-		// int index = indexOptional.orElseGet(() -> extractValue(keyLiteral, Integer.class));
-		// Because the logical in `orElseGet` always executed no matter whether indexOptional is present or not.
-		int index;
-		if (indexOptional.isPresent()) {
-			index = indexOptional.get();
-		} else {
-			index = extractValue(keyLiteral, Integer.class);
-		}
-		return relBuilder.getRexBuilder().makeFieldAccess(child, index);
-	}
-
-	private RexNode convertIn(List<Expression> children) {
-		Expression headExpr = children.get(1);
-		if (headExpr instanceof TableReferenceExpression) {
-			QueryOperation tableOperation = ((TableReferenceExpression) headExpr).getQueryOperation();
-			RexNode child = children.get(0).accept(this);
-			return RexSubQuery.in(
-					((FlinkRelBuilder) relBuilder).queryOperation(tableOperation).build(),
-					ImmutableList.of(child));
-		} else {
-			List<RexNode> child = convertCallChildren(children);
-			return relBuilder.call(FlinkSqlOperatorTable.IN, child);
-		}
-	}
-
-	private RexNode convertReinterpretCast(List<Expression> children) {
-		RexNode child = children.get(0).accept(this);
-		TypeLiteralExpression type = (TypeLiteralExpression) children.get(1);
-		RexNode checkOverflow = children.get(2).accept(this);
-		return relBuilder.getRexBuilder().makeReinterpretCast(
-				typeFactory.createFieldTypeFromLogicalType(
-						type.getOutputDataType().getLogicalType().copy(child.getType().isNullable())),
-				child,
-				checkOverflow);
-	}
-
-	private RexNode convertCast(List<Expression> children) {
-		RexNode child = children.get(0).accept(this);
-		TypeLiteralExpression type = (TypeLiteralExpression) children.get(1);
-		return relBuilder.getRexBuilder().makeAbstractCast(
-				typeFactory.createFieldTypeFromLogicalType(
-						type.getOutputDataType().getLogicalType().copy(child.getType().isNullable())),
-				child);
-	}
-
-	@Override
-	public RexNode visit(ValueLiteralExpression valueLiteral) {
-		LogicalType type = fromDataTypeToLogicalType(valueLiteral.getOutputDataType());
-		RexBuilder rexBuilder = relBuilder.getRexBuilder();
-		FlinkTypeFactory typeFactory = (FlinkTypeFactory) relBuilder.getTypeFactory();
-		if (valueLiteral.isNull()) {
-			return relBuilder.getRexBuilder()
-					.makeCast(
-							typeFactory.createFieldTypeFromLogicalType(type),
-							relBuilder.getRexBuilder().constantNull());
-		}
-
-		switch (type.getTypeRoot()) {
-			case DECIMAL:
-				DecimalType dt = (DecimalType) type;
-				BigDecimal bigDecimal = extractValue(valueLiteral, BigDecimal.class);
-				RelDataType decType = relBuilder.getTypeFactory().createSqlType(SqlTypeName.DECIMAL,
-						dt.getPrecision(), dt.getScale());
-				return relBuilder.getRexBuilder().makeExactLiteral(bigDecimal, decType);
-			case BIGINT:
-				// create BIGINT literals for long type
-				BigDecimal bigint = extractValue(valueLiteral, BigDecimal.class);
-				return relBuilder.getRexBuilder().makeBigintLiteral(bigint);
-			case FLOAT:
-				//Float/Double type should be liked as java type here.
-				return relBuilder.getRexBuilder().makeApproxLiteral(
-						extractValue(valueLiteral, BigDecimal.class),
-						relBuilder.getTypeFactory().createSqlType(SqlTypeName.FLOAT));
-			case DOUBLE:
-				//Float/Double type should be liked as java type here.
-				return rexBuilder.makeApproxLiteral(
-						extractValue(valueLiteral, BigDecimal.class),
-						relBuilder.getTypeFactory().createSqlType(SqlTypeName.DOUBLE));
-			case DATE:
-				return relBuilder.getRexBuilder().makeDateLiteral(DateString.fromCalendarFields(
-						valueAsCalendar(extractValue(valueLiteral, java.sql.Date.class))));
-			case TIME_WITHOUT_TIME_ZONE:
-				return relBuilder.getRexBuilder().makeTimeLiteral(TimeString.fromCalendarFields(
-						valueAsCalendar(extractValue(valueLiteral, java.sql.Time.class))), 0);
-			case TIMESTAMP_WITHOUT_TIME_ZONE:
-				return relBuilder.getRexBuilder().makeTimestampLiteral(TimestampString.fromCalendarFields(
-						valueAsCalendar(extractValue(valueLiteral, java.sql.Timestamp.class))), 3);
-			case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-				TimeZone timeZone = TimeZone.getTimeZone(((FlinkContext) ((FlinkRelBuilder) this.relBuilder)
-						.getCluster().getPlanner().getContext()).getTableConfig().getLocalTimeZone());
-				return this.relBuilder.getRexBuilder().makeTimestampWithLocalTimeZoneLiteral(
-						new TimestampWithTimeZoneString(
-								TimestampString.fromMillisSinceEpoch(
-										extractValue(valueLiteral, java.time.Instant.class).toEpochMilli()),
-								timeZone)
-								.withTimeZone(DateTimeUtils.UTC_ZONE)
-								.getLocalTimestampString(), 3);
-			case INTERVAL_YEAR_MONTH:
-				return this.relBuilder.getRexBuilder().makeIntervalLiteral(
-						BigDecimal.valueOf(extractValue(valueLiteral, Integer.class)),
-						new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO));
-			case INTERVAL_DAY_TIME:
-				return this.relBuilder.getRexBuilder().makeIntervalLiteral(
-						BigDecimal.valueOf(extractValue(valueLiteral, Long.class)),
-						new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO));
-			default:
-				break;
-		}
-		Object object = extractValue(valueLiteral, Object.class);
-		if (object instanceof TimePointUnit) {
-			TimeUnit value;
-			switch ((TimePointUnit) object) {
-				case YEAR:
-					value = TimeUnit.YEAR;
-					break;
-				case MONTH:
-					value = TimeUnit.MONTH;
-					break;
-				case DAY:
-					value = TimeUnit.DAY;
-					break;
-				case HOUR:
-					value = TimeUnit.HOUR;
-					break;
-				case MINUTE:
-					value = TimeUnit.MINUTE;
-					break;
-				case SECOND:
-					value = TimeUnit.SECOND;
-					break;
-				case QUARTER:
-					value = TimeUnit.QUARTER;
-					break;
-				case WEEK:
-					value = TimeUnit.WEEK;
-					break;
-				case MILLISECOND:
-					value = TimeUnit.MILLISECOND;
-					break;
-				case MICROSECOND:
-					value = TimeUnit.MICROSECOND;
-					break;
-				default:
-					throw new UnsupportedOperationException();
-			}
-			return relBuilder.getRexBuilder().makeFlag(value);
-		} else if (object instanceof TimeIntervalUnit) {
-			TimeUnitRange value;
-			switch ((TimeIntervalUnit) object) {
-				case YEAR:
-					value = TimeUnitRange.YEAR;
-					break;
-				case YEAR_TO_MONTH:
-					value = TimeUnitRange.YEAR_TO_MONTH;
-					break;
-				case QUARTER:
-					value = TimeUnitRange.QUARTER;
-					break;
-				case MONTH:
-					value = TimeUnitRange.MONTH;
-					break;
-				case WEEK:
-					value = TimeUnitRange.WEEK;
-					break;
-				case DAY:
-					value = TimeUnitRange.DAY;
-					break;
-				case DAY_TO_HOUR:
-					value = TimeUnitRange.DAY_TO_HOUR;
-					break;
-				case DAY_TO_MINUTE:
-					value = TimeUnitRange.DAY_TO_MINUTE;
-					break;
-				case DAY_TO_SECOND:
-					value = TimeUnitRange.DAY_TO_SECOND;
-					break;
-				case HOUR:
-					value = TimeUnitRange.HOUR;
-					break;
-				case SECOND:
-					value = TimeUnitRange.SECOND;
-					break;
-				case HOUR_TO_MINUTE:
-					value = TimeUnitRange.HOUR_TO_MINUTE;
-					break;
-				case HOUR_TO_SECOND:
-					value = TimeUnitRange.HOUR_TO_SECOND;
-					break;
-				case MINUTE:
-					value = TimeUnitRange.MINUTE;
-					break;
-				case MINUTE_TO_SECOND:
-					value = TimeUnitRange.MINUTE_TO_SECOND;
-					break;
-				default:
-					throw new UnsupportedOperationException();
-			}
-			return relBuilder.getRexBuilder().makeFlag(value);
-		} else {
-			return relBuilder.literal(extractValue(valueLiteral, Object.class));
-		}
-	}
-
-	/**
-	 * Extracts a value from a literal. Including planner-specific instances such as {@link Decimal}.
-	 */
-	@SuppressWarnings("unchecked")
-	private static <T> T extractValue(ValueLiteralExpression literal, Class<T> clazz) {
-		final Optional<Object> possibleObject = literal.getValueAs(Object.class);
-		if (!possibleObject.isPresent()) {
-			throw new TableException("Invalid literal.");
-		}
-		final Object object = possibleObject.get();
-
-		if (clazz.equals(BigDecimal.class)) {
-			final Optional<BigDecimal> possibleDecimal = literal.getValueAs(BigDecimal.class);
-			if (possibleDecimal.isPresent()) {
-				return (T) possibleDecimal.get();
-			}
-			if (object instanceof Decimal) {
-				return (T) ((Decimal) object).toBigDecimal();
-			}
-		}
-
-		return literal.getValueAs(clazz)
-				.orElseThrow(() -> new TableException("Unsupported literal class: " + clazz));
-	}
-
-	/**
-	 * Convert a Date value to a Calendar. Calcite's fromCalendarField functions use the
-	 * Calendar.get methods, so the raw values of the individual fields are preserved when
-	 * converted to the String formats.
-	 *
-	 * @return get the Calendar value
-	 */
-	private static Calendar valueAsCalendar(Object value) {
-		Date date = (Date) value;
-		Calendar cal = Calendar.getInstance();
-		cal.setTime(date);
-		return cal;
-	}
-
-	@Override
-	public RexNode visit(FieldReferenceExpression fieldReference) {
-		return relBuilder.field(fieldReference.getName());
-	}
-
-	@Override
-	public RexNode visit(TypeLiteralExpression typeLiteral) {
-		throw new UnsupportedOperationException();
-	}
-
-	@Override
-	public RexNode visit(Expression other) {
-		if (other instanceof UnresolvedReferenceExpression) {
-			return visitUnresolvedReferenceExpression((UnresolvedReferenceExpression) other);
-		} else if (other instanceof ResolvedAggInputReference) {
-			return visitResolvedAggInputReference((ResolvedAggInputReference) other);
-		} else if (other instanceof ResolvedAggLocalReference) {
-			return visitResolvedAggLocalReference((ResolvedAggLocalReference) other);
-		} else if (other instanceof ResolvedDistinctKeyReference) {
-			return visitResolvedDistinctKeyReference((ResolvedDistinctKeyReference) other);
-		} else if (other instanceof UnresolvedCallExpression) {
-			return visit((UnresolvedCallExpression) other);
-		} else if (other instanceof RexNodeExpression) {
-			return ((RexNodeExpression) other).getRexNode();
-		} else {
-			throw new UnsupportedOperationException(other.getClass().getSimpleName() + ":" + other.toString());
-		}
-	}
-
-	private RexNode visitUnresolvedReferenceExpression(UnresolvedReferenceExpression field) {
-		return relBuilder.field(field.getName());
-	}
-
-	private RexNode visitResolvedAggInputReference(ResolvedAggInputReference reference) {
-		// using index to resolve field directly, name used in toString only
-		return new RexInputRef(
-				reference.getIndex(),
-				typeFactory.createFieldTypeFromLogicalType(reference.getResultType()));
-	}
-
-	private RexNode visitResolvedAggLocalReference(ResolvedAggLocalReference reference) {
-		LogicalType type = reference.getResultType();
-		return new RexAggLocalVariable(
-				reference.getFieldTerm(),
-				reference.getNullTerm(),
-				typeFactory.createFieldTypeFromLogicalType(type),
-				type);
-	}
-
-	private RexNode visitResolvedDistinctKeyReference(ResolvedDistinctKeyReference reference) {
-		LogicalType type = reference.getResultType();
-		return new RexDistinctKeyVariable(
-				reference.getName(),
-				typeFactory.createFieldTypeFromLogicalType(type),
-				type);
-	}
-
-	private RexNode visit(UnresolvedCallExpression call) {
-		FunctionDefinition func = call.getFunctionDefinition();
-		switch (func.getKind()) {
-			case SCALAR:
-				if (func instanceof ScalarFunctionDefinition) {
-					ScalarFunction scalaFunc = ((ScalarFunctionDefinition) func).getScalarFunction();
-					List<RexNode> child = convertCallChildren(call.getChildren());
-					SqlFunction sqlFunction = UserDefinedFunctionUtils.createScalarSqlFunction(
-							scalaFunc.functionIdentifier(),
-							scalaFunc.toString(),
-							scalaFunc,
-							typeFactory);
-					return relBuilder.call(sqlFunction, child);
-				} else {
-					FunctionDefinition def = call.getFunctionDefinition();
-					if (conversionsOfBuiltInFunc.containsKey(def)) {
-						RexNodeConversion conversion = conversionsOfBuiltInFunc.get(def);
-						return conversion.convert(call);
-					} else {
-						throw new UnsupportedOperationException(def.toString());
-					}
-				}
-
-			default:
-				throw new UnsupportedOperationException();
-		}
-	}
-
-	private RexNode createCollation(RexNode node, RelFieldCollation.Direction direction,
-			RelFieldCollation.NullDirection nullDirection, Set<SqlKind> kinds) {
-		switch (node.getKind()) {
-			case DESCENDING:
-				kinds.add(node.getKind());
-				return createCollation(((RexCall) node).getOperands().get(0), RelFieldCollation.Direction.DESCENDING,
-						nullDirection, kinds);
-			case NULLS_FIRST:
-				kinds.add(node.getKind());
-				return createCollation(((RexCall) node).getOperands().get(0), direction,
-						RelFieldCollation.NullDirection.FIRST, kinds);
-			case NULLS_LAST:
-				kinds.add(node.getKind());
-				return createCollation(((RexCall) node).getOperands().get(0), direction,
-						RelFieldCollation.NullDirection.LAST, kinds);
-			default:
-				if (nullDirection == null) {
-					// Set the null direction if not specified.
-					// Consistent with HIVE/SPARK/MYSQL/BLINK-RUNTIME.
-					if (FlinkPlannerImpl.defaultNullCollation()
-							.last(direction.equals(RelFieldCollation.Direction.DESCENDING))) {
-						kinds.add(SqlKind.NULLS_LAST);
-					} else {
-						kinds.add(SqlKind.NULLS_FIRST);
-					}
-				}
-				return node;
-		}
-	}
-
-	private RexWindowBound createBound(Expression bound, SqlKind sqlKind) {
-		if (bound instanceof CallExpression) {
-			CallExpression callExpr = (CallExpression) bound;
-			FunctionDefinition func = callExpr.getFunctionDefinition();
-			if (BuiltInFunctionDefinitions.UNBOUNDED_ROW.equals(func) || BuiltInFunctionDefinitions.UNBOUNDED_RANGE
-					.equals(func)) {
-				SqlNode unbounded = sqlKind.equals(SqlKind.PRECEDING) ? SqlWindow
-						.createUnboundedPreceding(SqlParserPos.ZERO) :
-						SqlWindow.createUnboundedFollowing(SqlParserPos.ZERO);
-				return RexWindowBound.create(unbounded, null);
-			} else if (BuiltInFunctionDefinitions.CURRENT_ROW.equals(func) || BuiltInFunctionDefinitions.CURRENT_RANGE
-					.equals(func)) {
-				SqlNode currentRow = SqlWindow.createCurrentRow(SqlParserPos.ZERO);
-				return RexWindowBound.create(currentRow, null);
-			} else {
-				throw new IllegalArgumentException("Unexpected expression: " + bound);
-			}
-		} else if (bound instanceof ValueLiteralExpression) {
-			RelDataType returnType = typeFactory
-					.createFieldTypeFromLogicalType(new DecimalType(true, 19, 0));
-			SqlOperator sqlOperator = new SqlPostfixOperator(
-					sqlKind.name(),
-					sqlKind,
-					2,
-					new OrdinalReturnTypeInference(0),
-					null,
-					null);
-			SqlNode[] operands = new SqlNode[] { SqlLiteral.createExactNumeric("1", SqlParserPos.ZERO) };
-			SqlNode node = new SqlBasicCall(sqlOperator, operands, SqlParserPos.ZERO);
-
-			ValueLiteralExpression literalExpr = (ValueLiteralExpression) bound;
-			RexNode literalRexNode = literalExpr.getValueAs(Double.class).map(
-					v -> relBuilder.literal(BigDecimal.valueOf((Double) v))).orElse(
-					relBuilder.literal(extractValue(literalExpr, Object.class)));
-
-			List<RexNode> expressions = new ArrayList<>();
-			expressions.add(literalRexNode);
-			RexNode rexNode = relBuilder.getRexBuilder().makeCall(returnType, sqlOperator, expressions);
-			return RexWindowBound.create(node, rexNode);
-		} else {
-			throw new TableException("Unexpected expression: " + bound);
-		}
-	}
-
-	/**
-	 * RexNodeConversion to define how to convert a {@link CallExpression} or a {@link UnresolvedCallExpression} which
-	 * has built-in FunctionDefinition to RexNode.
-	 */
-	private interface RexNodeConversion {
-
-		RexNode convert(List<Expression> children);
-
-		default RexNode convert(CallExpression expression) {
-			return convert(expression.getChildren());
-		}
-
-		default RexNode convert(UnresolvedCallExpression unresolvedCallExpression) {
-			return convert(unresolvedCallExpression.getChildren());
-		}
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeExpression.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeExpression.java
deleted file mode 100644
index 8c33bc4..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/RexNodeExpression.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.types.DataType;
-
-import org.apache.calcite.rex.RexNode;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Dummy wrapper for expressions that were converted to RexNode in a different way.
- */
-public class RexNodeExpression implements ResolvedExpression {
-
-	private RexNode rexNode;
-	private DataType outputDataType;
-
-	public RexNodeExpression(RexNode rexNode, DataType outputDataType) {
-		this.rexNode = rexNode;
-		this.outputDataType = outputDataType;
-	}
-
-	public RexNode getRexNode() {
-		return rexNode;
-	}
-
-	@Override
-	public String asSummaryString() {
-		return rexNode.toString();
-	}
-
-	@Override
-	public List<Expression> getChildren() {
-		return Collections.emptyList();
-	}
-
-	@Override
-	public <R> R accept(ExpressionVisitor<R> visitor) {
-		return visitor.visit(this);
-	}
-
-	@Override
-	public DataType getOutputDataType() {
-		return outputDataType;
-	}
-
-	@Override
-	public List<ResolvedExpression> getResolvedChildren() {
-		return new ArrayList<>();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/SqlAggFunctionVisitor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/SqlAggFunctionVisitor.java
deleted file mode 100644
index 8c8d43c..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/expressions/SqlAggFunctionVisitor.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.expressions;
-
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.calcite.FlinkTypeFactory;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.functions.AggregateFunctionDefinition;
-import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
-import org.apache.flink.table.functions.FunctionDefinition;
-import org.apache.flink.table.functions.UserDefinedAggregateFunction;
-import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable;
-import org.apache.flink.table.functions.utils.AggSqlFunction;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.calcite.sql.SqlAggFunction;
-
-import java.util.IdentityHashMap;
-import java.util.Map;
-
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.isFunctionOfKind;
-import static org.apache.flink.table.functions.FunctionKind.AGGREGATE;
-import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType;
-
-/**
- * The class to get {@link SqlAggFunctionVisitor} of CallExpression.
- */
-public class SqlAggFunctionVisitor extends ExpressionDefaultVisitor<SqlAggFunction> {
-
-	private static final Map<FunctionDefinition, SqlAggFunction> AGG_DEF_SQL_OPERATOR_MAPPING = new IdentityHashMap<>();
-
-	static {
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.AVG, FlinkSqlOperatorTable.AVG);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.COUNT, FlinkSqlOperatorTable.COUNT);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.MAX, FlinkSqlOperatorTable.MAX);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.MIN, FlinkSqlOperatorTable.MIN);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.SUM, FlinkSqlOperatorTable.SUM);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.SUM0, FlinkSqlOperatorTable.SUM0);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.STDDEV_POP, FlinkSqlOperatorTable.STDDEV_POP);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.STDDEV_SAMP, FlinkSqlOperatorTable.STDDEV_SAMP);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.VAR_POP, FlinkSqlOperatorTable.VAR_POP);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.VAR_SAMP, FlinkSqlOperatorTable.VAR_SAMP);
-		AGG_DEF_SQL_OPERATOR_MAPPING.put(BuiltInFunctionDefinitions.COLLECT, FlinkSqlOperatorTable.COLLECT);
-	}
-
-	private final FlinkTypeFactory typeFactory;
-
-	public SqlAggFunctionVisitor(FlinkTypeFactory typeFactory) {
-		this.typeFactory = typeFactory;
-	}
-
-	@Override
-	public SqlAggFunction visit(CallExpression call) {
-		Preconditions.checkArgument(isFunctionOfKind(call, AGGREGATE));
-		FunctionDefinition def = call.getFunctionDefinition();
-		if (AGG_DEF_SQL_OPERATOR_MAPPING.containsKey(def)) {
-			return AGG_DEF_SQL_OPERATOR_MAPPING.get(def);
-		}
-		if (BuiltInFunctionDefinitions.DISTINCT == def) {
-			Expression innerAgg = call.getChildren().get(0);
-			return innerAgg.accept(this);
-		}
-		AggregateFunctionDefinition aggDef = (AggregateFunctionDefinition) def;
-		UserDefinedAggregateFunction userDefinedAggregateFunc = aggDef.getAggregateFunction();
-		if (userDefinedAggregateFunc instanceof AggregateFunction) {
-			AggregateFunction aggFunc = (AggregateFunction) userDefinedAggregateFunc;
-			return new AggSqlFunction(
-					aggFunc.functionIdentifier(),
-					aggFunc.toString(),
-					aggFunc,
-					fromLegacyInfoToDataType(aggDef.getResultTypeInfo()),
-					fromLegacyInfoToDataType(aggDef.getAccumulatorTypeInfo()),
-					typeFactory,
-					aggFunc.requiresOver());
-		} else {
-			throw new UnsupportedOperationException("TableAggregateFunction is not supported yet!");
-		}
-	}
-
-	@Override
-	protected SqlAggFunction defaultMethod(Expression expression) {
-		throw new TableException("Unexpected expression: " + expression);
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/InternalFunctionDefinitions.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/InternalFunctionDefinitions.java
deleted file mode 100644
index c9d89e2..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/InternalFunctionDefinitions.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions;
-
-import org.apache.flink.table.types.inference.TypeStrategies;
-
-import static org.apache.flink.table.functions.FunctionKind.SCALAR;
-
-/**
- * Dictionary of function definitions for all internal used functions.
- */
-public class InternalFunctionDefinitions {
-
-	public static final BuiltInFunctionDefinition THROW_EXCEPTION =
-		new BuiltInFunctionDefinition.Builder()
-			.name("throwException")
-			.kind(SCALAR)
-			.outputTypeStrategy(TypeStrategies.MISSING)
-			.build();
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.java
deleted file mode 100644
index 8150843..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.calcite.FlinkTypeSystem;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-
-import java.math.BigDecimal;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.div;
-import static org.apache.flink.table.expressions.ExpressionBuilder.equalTo;
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in avg aggregate function.
- */
-public abstract class AvgAggFunction extends DeclarativeAggregateFunction {
-
-	private UnresolvedReferenceExpression sum = unresolvedRef("sum");
-	private UnresolvedReferenceExpression count = unresolvedRef("count");
-
-	public abstract DataType getSumType();
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] {
-				sum,
-				count};
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] {
-				getSumType(),
-				DataTypes.BIGINT()
-		};
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* sum = */ literal(0L, getSumType()),
-				/* count = */ literal(0L)};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* sum = */ ifThenElse(isNull(operand(0)), sum, plus(sum, operand(0))),
-				/* count = */ ifThenElse(isNull(operand(0)), count, plus(count, literal(1L))),
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		return new Expression[] {
-				/* sum = */ ifThenElse(isNull(operand(0)), sum, minus(sum, operand(0))),
-				/* count = */ ifThenElse(isNull(operand(0)), count, minus(count, literal(1L))),
-		};
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* sum = */ plus(sum, mergeOperand(sum)),
-				/* count = */ plus(count, mergeOperand(count))
-		};
-	}
-
-	/**
-	 * If all input are nulls, count will be 0 and we will get null after the division.
-	 */
-	@Override
-	public Expression getValueExpression() {
-		return ifThenElse(equalTo(count, literal(0L)), nullOf(getResultType()), div(sum, count));
-	}
-
-	/**
-	 * Built-in Int Avg aggregate function for integral arguments,
-	 * including BYTE, SHORT, INT, LONG.
-	 * The result type is DOUBLE.
-	 */
-	public static class IntegralAvgAggFunction extends AvgAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-
-		@Override
-		public DataType getSumType() {
-			return DataTypes.BIGINT();
-		}
-	}
-
-	/**
-	 * Built-in Double Avg aggregate function.
-	 */
-	public static class DoubleAvgAggFunction extends AvgAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-
-		@Override
-		public DataType getSumType() {
-			return DataTypes.DOUBLE();
-		}
-
-		@Override
-		public Expression[] initialValuesExpressions() {
-			return new Expression[] {literal(0D), literal(0L)};
-		}
-	}
-
-	/**
-	 * Built-in Decimal Avg aggregate function.
-	 */
-	public static class DecimalAvgAggFunction extends AvgAggFunction {
-
-		private final DecimalType type;
-
-		public DecimalAvgAggFunction(DecimalType type) {
-			this.type = type;
-		}
-
-		@Override
-		public DataType getResultType() {
-			DecimalType t = FlinkTypeSystem.inferAggAvgType(type.getScale());
-			return DataTypes.DECIMAL(t.getPrecision(), t.getScale());
-		}
-
-		@Override
-		public DataType getSumType() {
-			DecimalType t = FlinkTypeSystem.inferAggSumType(type.getScale());
-			return DataTypes.DECIMAL(t.getPrecision(), t.getScale());
-		}
-
-		@Override
-		public Expression[] initialValuesExpressions() {
-			return new Expression[] {
-				literal(
-					BigDecimal.ZERO,
-					getSumType()),
-				literal(0L)
-			};
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CollectAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CollectAggFunction.java
deleted file mode 100644
index 4d7b8bd..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CollectAggFunction.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.MapTypeInfo;
-import org.apache.flink.table.api.dataview.MapView;
-import org.apache.flink.table.functions.AggregateFunction;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * Aggregate function for COLLECT.
- * @param <T> type of collect element.
- */
-public class CollectAggFunction<T>
-	extends AggregateFunction<Map<T, Integer>, CollectAggFunction.CollectAccumulator<T>> {
-
-	private static final long serialVersionUID = -5860934997657147836L;
-
-	private final TypeInformation<T> elementType;
-
-	public CollectAggFunction(TypeInformation<T> elementType) {
-		this.elementType = elementType;
-	}
-
-	/** The initial accumulator for Collect aggregate function. */
-	public static class CollectAccumulator<T> {
-		public MapView<T, Integer> map = null;
-
-		@Override
-		public boolean equals(Object o) {
-			if (this == o) {
-				return true;
-			}
-			if (o == null || getClass() != o.getClass()) {
-				return false;
-			}
-			CollectAccumulator<?> that = (CollectAccumulator<?>) o;
-			return Objects.equals(map, that.map);
-		}
-	}
-
-	public CollectAccumulator<T> createAccumulator() {
-		CollectAccumulator<T> acc = new CollectAccumulator<>();
-		acc.map = new MapView<>(elementType, Types.INT);
-		return acc;
-	}
-
-	public void resetAccumulator(CollectAccumulator<T> accumulator) {
-		accumulator.map.clear();
-	}
-
-	public void accumulate(CollectAccumulator<T> accumulator, T value) throws Exception {
-		if (value != null) {
-			Integer count = accumulator.map.get(value);
-			if (count != null) {
-				accumulator.map.put(value, count + 1);
-			} else {
-				accumulator.map.put(value, 1);
-			}
-		}
-	}
-
-	public void retract(CollectAccumulator<T> accumulator, T value) throws Exception {
-		if (value != null) {
-			Integer count = accumulator.map.get(value);
-			if (count != null) {
-				if (count == 1) {
-					accumulator.map.remove(value);
-				} else {
-					accumulator.map.put(value, count - 1);
-				}
-			} else {
-				accumulator.map.put(value, -1);
-			}
-		}
-	}
-
-	public void merge(CollectAccumulator<T> accumulator, Iterable<CollectAccumulator<T>> others) throws Exception {
-		for (CollectAccumulator<T> other : others) {
-			for (Map.Entry<T, Integer> entry : other.map.entries()) {
-				T key = entry.getKey();
-				Integer newCount = entry.getValue();
-				Integer oldCount = accumulator.map.get(key);
-				if (oldCount == null) {
-					accumulator.map.put(key, newCount);
-				} else {
-					accumulator.map.put(key, oldCount + newCount);
-				}
-			}
-		}
-	}
-
-	@Override
-	public Map<T, Integer> getValue(CollectAccumulator<T> accumulator) {
-		Map<T, Integer> result = new HashMap<>();
-		try {
-			for (Map.Entry<T, Integer> entry : accumulator.map.entries()) {
-				result.put(entry.getKey(), entry.getValue());
-			}
-			return result;
-		} catch (Exception e) {
-			throw new RuntimeException(e);
-		}
-	}
-
-	@Override
-	public TypeInformation<Map<T, Integer>> getResultType() {
-		return new MapTypeInfo<>(elementType, Types.INT);
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatAggFunction.java
deleted file mode 100644
index 19cc878..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatAggFunction.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.concat;
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in concat aggregate function.
- */
-public class ConcatAggFunction extends DeclarativeAggregateFunction {
-	private int operandCount;
-	private UnresolvedReferenceExpression acc = unresolvedRef("concatAcc");
-	private UnresolvedReferenceExpression accDelimiter = unresolvedRef("accDelimiter");
-	private Expression delimiter;
-	private Expression operand;
-
-	public ConcatAggFunction(int operandCount) {
-		this.operandCount = operandCount;
-		if (operandCount == 1) {
-			delimiter = literal("\n", DataTypes.STRING());
-			operand = operand(0);
-		} else {
-			delimiter = operand(0);
-			operand = operand(1);
-		}
-	}
-
-	@Override
-	public int operandCount() {
-		return operandCount;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { accDelimiter, acc };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { DataTypes.STRING(), DataTypes.STRING() };
-	}
-
-	@Override
-	public DataType getResultType() {
-		return DataTypes.STRING();
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* delimiter */ literal("\n", DataTypes.STRING()),
-				/* acc */ nullOf(DataTypes.STRING())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* delimiter */
-				delimiter,
-				/* acc */
-				ifThenElse(isNull(operand), acc,
-						ifThenElse(isNull(acc), operand, concat(concat(acc, delimiter), operand)))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		throw new TableException("This function does not support retraction.");
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* delimiter */
-				mergeOperand(accDelimiter),
-				/* acc */
-				ifThenElse(isNull(mergeOperand(acc)), acc,
-						ifThenElse(isNull(acc), mergeOperand(acc),
-								concat(concat(acc, mergeOperand(accDelimiter)), mergeOperand(acc))))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return acc;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
deleted file mode 100644
index 8d2c7ca..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWithRetractAggFunction.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.table.api.dataview.ListView;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.BinaryStringUtil;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-/**
- * built-in concat with retraction aggregate function.
- */
-public final class ConcatWithRetractAggFunction
-	extends AggregateFunction<BinaryString, ConcatWithRetractAggFunction.ConcatWithRetractAccumulator> {
-
-	private static final long serialVersionUID = -2836795091288790955L;
-	private static final BinaryString lineDelimiter = BinaryString.fromString("\n");
-
-	/**
-	 * The initial accumulator for concat with retraction aggregate function.
-	 */
-	public static class ConcatWithRetractAccumulator {
-		public ListView<BinaryString> list = new ListView<>(BinaryStringTypeInfo.INSTANCE);
-		public ListView<BinaryString> retractList = new ListView<>(BinaryStringTypeInfo.INSTANCE);
-
-		@VisibleForTesting
-		@Override
-		public boolean equals(Object o) {
-			if (this == o) {
-				return true;
-			}
-			if (o == null || getClass() != o.getClass()) {
-				return false;
-			}
-			ConcatWithRetractAccumulator that = (ConcatWithRetractAccumulator) o;
-			return Objects.equals(list, that.list) &&
-				Objects.equals(retractList, that.retractList);
-		}
-	}
-
-	@Override
-	public ConcatWithRetractAccumulator createAccumulator() {
-		return new ConcatWithRetractAccumulator();
-	}
-
-	public void accumulate(ConcatWithRetractAccumulator acc, BinaryString value) throws Exception {
-		// ignore null value
-		if (value != null) {
-			acc.list.add(value);
-		}
-	}
-
-	public void retract(ConcatWithRetractAccumulator acc, BinaryString value) throws Exception {
-		if (value != null) {
-			if (!acc.list.remove(value)) {
-				acc.retractList.add(value);
-			}
-		}
-	}
-
-	public void merge(ConcatWithRetractAccumulator acc, Iterable<ConcatWithRetractAccumulator> its) throws Exception {
-		for (ConcatWithRetractAccumulator otherAcc : its) {
-			// merge list of acc and other
-			List<BinaryString> buffer = new ArrayList<>();
-			for (BinaryString binaryString : acc.list.get()) {
-				buffer.add(binaryString);
-			}
-			for (BinaryString binaryString : otherAcc.list.get()) {
-				buffer.add(binaryString);
-			}
-			// merge retract list of acc and other
-			List<BinaryString> retractBuffer = new ArrayList<>();
-			for (BinaryString binaryString : acc.retractList.get()) {
-				retractBuffer.add(binaryString);
-			}
-			for (BinaryString binaryString : otherAcc.retractList.get()) {
-				retractBuffer.add(binaryString);
-			}
-
-			// merge list & retract list
-			List<BinaryString> newRetractBuffer = new ArrayList<>();
-			for (BinaryString binaryString : retractBuffer) {
-				if (!buffer.remove(binaryString)) {
-					newRetractBuffer.add(binaryString);
-				}
-			}
-
-			// update to acc
-			acc.list.clear();
-			acc.list.addAll(buffer);
-			acc.retractList.clear();
-			acc.retractList.addAll(newRetractBuffer);
-		}
-	}
-
-	@Override
-	public BinaryString getValue(ConcatWithRetractAccumulator acc) {
-		try {
-			Iterable<BinaryString> accList = acc.list.get();
-			if (accList == null || !accList.iterator().hasNext()) {
-				// return null when the list is empty
-				return null;
-			} else {
-				return BinaryStringUtil.concatWs(lineDelimiter, accList);
-			}
-		} catch (Exception e) {
-			throw new FlinkRuntimeException(e);
-		}
-	}
-
-	public void resetAccumulator(ConcatWithRetractAccumulator acc) {
-		acc.list.clear();
-		acc.retractList.clear();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
deleted file mode 100644
index 23cc4cc..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/ConcatWsWithRetractAggFunction.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.table.api.dataview.ListView;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.BinaryStringUtil;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Objects;
-
-/**
- * built-in concatWs with retraction aggregate function.
- */
-public final class ConcatWsWithRetractAggFunction
-	extends AggregateFunction<BinaryString, ConcatWsWithRetractAggFunction.ConcatWsWithRetractAccumulator> {
-
-	private static final long serialVersionUID = -8627988150350160473L;
-
-	/**
-	 * The initial accumulator for concat with retraction aggregate function.
-	 */
-	public static class ConcatWsWithRetractAccumulator {
-		public ListView<BinaryString> list = new ListView<>(BinaryStringTypeInfo.INSTANCE);
-		public ListView<BinaryString> retractList = new ListView<>(BinaryStringTypeInfo.INSTANCE);
-		public BinaryString delimiter = BinaryString.fromString("\n");
-
-		@VisibleForTesting
-		@Override
-		public boolean equals(Object o) {
-			if (this == o) {
-				return true;
-			}
-			if (o == null || getClass() != o.getClass()) {
-				return false;
-			}
-			ConcatWsWithRetractAccumulator that = (ConcatWsWithRetractAccumulator) o;
-			return Objects.equals(list, that.list) &&
-				Objects.equals(retractList, that.retractList) &&
-				Objects.equals(delimiter, that.delimiter);
-		}
-	}
-
-	@Override
-	public ConcatWsWithRetractAccumulator createAccumulator() {
-		return new ConcatWsWithRetractAccumulator();
-	}
-
-	public void accumulate(ConcatWsWithRetractAccumulator acc, BinaryString lineDelimiter, BinaryString value) throws Exception {
-		// ignore null value
-		if (value != null) {
-			acc.delimiter = lineDelimiter;
-			acc.list.add(value);
-		}
-	}
-
-	public void retract(ConcatWsWithRetractAccumulator acc, BinaryString lineDelimiter, BinaryString value) throws Exception {
-		if (value != null) {
-			acc.delimiter = lineDelimiter;
-			if (!acc.list.remove(value)) {
-				acc.retractList.add(value);
-			}
-		}
-	}
-
-	public void merge(ConcatWsWithRetractAccumulator acc, Iterable<ConcatWsWithRetractAccumulator> its) throws Exception {
-		for (ConcatWsWithRetractAccumulator otherAcc : its) {
-			if (!otherAcc.list.get().iterator().hasNext()
-				&& !otherAcc.retractList.get().iterator().hasNext()) {
-				// otherAcc is empty, skip it
-				continue;
-			}
-
-			acc.delimiter = otherAcc.delimiter;
-			// merge list of acc and other
-			List<BinaryString> buffer = new ArrayList<>();
-			for (BinaryString binaryString : acc.list.get()) {
-				buffer.add(binaryString);
-			}
-			for (BinaryString binaryString : otherAcc.list.get()) {
-				buffer.add(binaryString);
-			}
-			// merge retract list of acc and other
-			List<BinaryString> retractBuffer = new ArrayList<>();
-			for (BinaryString binaryString : acc.retractList.get()) {
-				retractBuffer.add(binaryString);
-			}
-			for (BinaryString binaryString : otherAcc.retractList.get()) {
-				retractBuffer.add(binaryString);
-			}
-
-			// merge list & retract list
-			List<BinaryString> newRetractBuffer = new ArrayList<>();
-			for (BinaryString binaryString : retractBuffer) {
-				if (!buffer.remove(binaryString)) {
-					newRetractBuffer.add(binaryString);
-				}
-			}
-
-			// update to acc
-			acc.list.clear();
-			acc.list.addAll(buffer);
-			acc.retractList.clear();
-			acc.retractList.addAll(newRetractBuffer);
-		}
-	}
-
-	@Override
-	public BinaryString getValue(ConcatWsWithRetractAccumulator acc) {
-		try {
-			Iterable<BinaryString> accList = acc.list.get();
-			if (accList == null || !accList.iterator().hasNext()) {
-				// return null when the list is empty
-				return null;
-			} else {
-				return BinaryStringUtil.concatWs(acc.delimiter, accList);
-			}
-		} catch (Exception e) {
-			throw new FlinkRuntimeException(e);
-		}
-	}
-
-	public void resetAccumulator(ConcatWsWithRetractAccumulator acc) {
-		acc.delimiter = BinaryString.fromString("\n");
-		acc.list.clear();
-		acc.retractList.clear();
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/Count1AggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/Count1AggFunction.java
deleted file mode 100644
index ca94f5e..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/Count1AggFunction.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * This count1 aggregate function returns the count1 of values
- * which go into it like [[CountAggFunction]].
- * It differs in that null values are also counted.
- */
-public class Count1AggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression count1 = unresolvedRef("count1");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { count1 };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { DataTypes.BIGINT() };
-	}
-
-	@Override
-	public DataType getResultType() {
-		return DataTypes.BIGINT();
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* count1 = */ literal(0L, getResultType())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* count1 = */ plus(count1, literal(1L))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		return new Expression[] {
-				/* count1 = */ minus(count1, literal(1L))
-		};
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* count1 = */ plus(count1, mergeOperand(count1))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return count1;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CountAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CountAggFunction.java
deleted file mode 100644
index a5a5639..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/CountAggFunction.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in count aggregate function.
- */
-public class CountAggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression count = unresolvedRef("count");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { count };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { DataTypes.BIGINT() };
-	}
-
-	@Override
-	public DataType getResultType() {
-		return DataTypes.BIGINT();
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* count = */ literal(0L, getResultType())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* count = */ ifThenElse(isNull(operand(0)), count, plus(count, literal(1L)))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		return new Expression[] {
-				/* count = */ ifThenElse(isNull(operand(0)), count, minus(count, literal(1L)))
-		};
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* count = */ plus(count, mergeOperand(count))
-		};
-	}
-
-	// If all input are nulls, count will be 0 and we will get result 0.
-	@Override
-	public Expression getValueExpression() {
-		return count;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DeclarativeAggregateFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DeclarativeAggregateFunction.java
deleted file mode 100644
index 04c10c1..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DeclarativeAggregateFunction.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.functions.FunctionKind;
-import org.apache.flink.table.functions.UserDefinedFunction;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.util.Preconditions;
-
-import java.util.Arrays;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * API for aggregation functions that are expressed in terms of expressions.
- *
- * <p>When implementing a new expression-based aggregate function, you should first decide how many
- * operands your function will have by implementing {@link #operandCount} method. And then you can
- * use {@link #operand} fields to represent your operand, like `operand(0)`, `operand(2)`.
- *
- * <p>Then you should declare all your buffer attributes by implementing
- * {@link #aggBufferAttributes}. You should declare all buffer attributes as
- * {@link UnresolvedReferenceExpression}, and make sure the name of your attributes are unique within
- * the function and it should not conflict with operandIndex. You can then use these attributes when
- * defining {@link #initialValuesExpressions}, {@link #accumulateExpressions},
- * {@link #mergeExpressions} and {@link #getValueExpression}.
- *
- * <p>See an full example: {@link AvgAggFunction}.
- */
-public abstract class DeclarativeAggregateFunction extends UserDefinedFunction {
-
-	private transient Set<String> aggBufferNamesCache;
-
-	/**
-	 * How many operands your function will deal with.
-	 */
-	public abstract int operandCount();
-
-	/**
-	 * All fields of the aggregate buffer.
-	 */
-	public abstract UnresolvedReferenceExpression[] aggBufferAttributes();
-
-	/**
-	 * All types of the aggregate buffer.
-	 */
-	public abstract DataType[] getAggBufferTypes();
-
-	/**
-	 * The result type of the function.
-	 */
-	public abstract DataType getResultType();
-
-	/**
-	 * Expressions for initializing empty aggregation buffers.
-	 */
-	public abstract Expression[] initialValuesExpressions();
-
-	/**
-	 * Expressions for accumulating the mutable aggregation buffer based on an input row.
-	 */
-	public abstract Expression[] accumulateExpressions();
-
-	/**
-	 * Expressions for retracting the mutable aggregation buffer based on an input row.
-	 */
-	public abstract Expression[] retractExpressions();
-
-	/**
-	 * A sequence of expressions for merging two aggregation buffers together. When defining these
-	 * expressions, you can use the syntax {@code attributeName} and
-	 * {@code mergeOperand(attributeName)} to refer to the attributes corresponding to each of
-	 * the buffers being merged.
-	 */
-	public abstract Expression[] mergeExpressions();
-
-	/**
-	 * An expression which returns the final value for this aggregate function.
-	 */
-	public abstract Expression getValueExpression();
-
-	private Set<String> getAggBufferNames() {
-		if (aggBufferNamesCache == null) {
-			aggBufferNamesCache = Arrays.stream(aggBufferAttributes())
-				.map(UnresolvedReferenceExpression::getName)
-				.collect(Collectors.toSet());
-		}
-		return aggBufferNamesCache;
-	}
-
-	private void validateOperandName(String name) {
-		if (getAggBufferNames().contains(name)) {
-			throw new IllegalStateException(
-				String.format("Agg buffer name(%s) should not same to operands.", name));
-		}
-	}
-
-	/**
-	 * Args of accumulate and retract, the input value (usually obtained from a new arrived data).
-	 */
-	public final UnresolvedReferenceExpression[] operands() {
-		int operandCount = operandCount();
-		Preconditions.checkState(operandCount >= 0, "inputCount must be greater than or equal to 0.");
-		UnresolvedReferenceExpression[] ret = new UnresolvedReferenceExpression[operandCount];
-		for (int i = 0; i < operandCount; i++) {
-			String name = String.valueOf(i);
-			validateOperandName(name);
-			ret[i] = unresolvedRef(name);
-		}
-		return ret;
-	}
-
-	/**
-	 * Arg of accumulate and retract, the input value (usually obtained from a new arrived data).
-	 */
-	public final UnresolvedReferenceExpression operand(int i) {
-		String name = String.valueOf(i);
-		if (getAggBufferNames().contains(name)) {
-			throw new IllegalStateException(
-				String.format("Agg buffer name(%s) should not same to operands.", name));
-		}
-		return unresolvedRef(name);
-	}
-
-	/**
-	 * Merge input of {@link #mergeExpressions()}, the input are AGG buffer generated by user definition.
-	 */
-	public final UnresolvedReferenceExpression mergeOperand(UnresolvedReferenceExpression aggBuffer) {
-		String name = String.valueOf(Arrays.asList(aggBufferAttributes()).indexOf(aggBuffer));
-		validateOperandName(name);
-		return unresolvedRef(name);
-	}
-
-	/**
-	 * Merge inputs of {@link #mergeExpressions()}, these inputs are agg buffer generated by user definition.
-	 */
-	public final UnresolvedReferenceExpression[] mergeOperands() {
-		UnresolvedReferenceExpression[] aggBuffers = aggBufferAttributes();
-		UnresolvedReferenceExpression[] ret = new UnresolvedReferenceExpression[aggBuffers.length];
-		for (int i = 0; i < aggBuffers.length; i++) {
-			String name = String.valueOf(i);
-			validateOperandName(name);
-			ret[i] = unresolvedRef(name);
-		}
-		return ret;
-	}
-
-	@Override
-	public final FunctionKind getKind() {
-		return FunctionKind.OTHER;
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DenseRankAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DenseRankAggFunction.java
deleted file mode 100644
index 0439746..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/DenseRankAggFunction.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.utils.LogicalTypeDataTypeConverter;
-
-import java.util.Arrays;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-
-/**
- * built-in dense_rank aggregate function.
- */
-public class DenseRankAggFunction extends RankLikeAggFunctionBase {
-
-	public DenseRankAggFunction(LogicalType[] orderKeyTypes) {
-		super(orderKeyTypes);
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		UnresolvedReferenceExpression[] aggBufferAttrs = new UnresolvedReferenceExpression[1 + lastValues.length];
-		aggBufferAttrs[0] = sequence;
-		System.arraycopy(lastValues, 0, aggBufferAttrs, 1, lastValues.length);
-		return aggBufferAttrs;
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		DataType[] aggBufferTypes = new DataType[1 + orderKeyTypes.length];
-		aggBufferTypes[0] = DataTypes.BIGINT();
-		System.arraycopy(Arrays.stream(orderKeyTypes)
-				.map(LogicalTypeDataTypeConverter::toDataType).toArray(DataType[]::new),
-				0, aggBufferTypes, 1, orderKeyTypes.length);
-		return aggBufferTypes;
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		Expression[] initExpressions = new Expression[1 + orderKeyTypes.length];
-		// sequence = 0L
-		initExpressions[0] = literal(0L);
-		for (int i = 0; i < orderKeyTypes.length; ++i) {
-			// lastValue_i = init value
-			initExpressions[i + 1] = generateInitLiteral(orderKeyTypes[i]);
-		}
-		return initExpressions;
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		Expression[] accExpressions = new Expression[1 + operands().length];
-		// sequence = if (lastValues equalTo orderKeys) sequence else sequence + 1
-		accExpressions[0] = ifThenElse(orderKeyEqualsExpression(), sequence, plus(sequence, literal(1L)));
-		Expression[] operands = operands();
-		System.arraycopy(operands, 0, accExpressions, 1, operands.length);
-		return accExpressions;
-	}
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
deleted file mode 100644
index a4fa317..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueAggFunction.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.dataformat.GenericRow;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-import org.apache.flink.table.types.logical.BigIntType;
-import org.apache.flink.table.types.logical.LogicalType;
-
-import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
-
-/**
- * built-in FirstValue aggregate function.
- */
-public abstract class FirstValueAggFunction<T> extends AggregateFunction<T, GenericRow> {
-
-	@Override
-	public boolean isDeterministic() {
-		return false;
-	}
-
-	@Override
-	public GenericRow createAccumulator() {
-		// The accumulator schema:
-		// firstValue: T
-		// firstOrder: Long
-		GenericRow acc = new GenericRow(2);
-		acc.setField(0, null);
-		acc.setLong(1, Long.MAX_VALUE);
-		return acc;
-	}
-
-	public void accumulate(GenericRow acc, Object value) {
-		if (value != null && acc.getLong(1) == Long.MAX_VALUE) {
-			acc.setField(0, value);
-			acc.setLong(1, System.currentTimeMillis());
-		}
-	}
-
-	public void accumulate(GenericRow acc, Object value, Long order) {
-		if (value != null && acc.getLong(1) > order) {
-			acc.setField(0, value);
-			acc.setLong(1, order);
-		}
-	}
-
-	public void resetAccumulator(GenericRow acc) {
-		acc.setField(0, null);
-		acc.setLong(1, Long.MAX_VALUE);
-	}
-
-	@Override
-	public T getValue(GenericRow acc) {
-		return (T) acc.getField(0);
-	}
-
-	@Override
-	public TypeInformation<GenericRow> getAccumulatorType() {
-		LogicalType[] fieldTypes = new LogicalType[] {
-				fromTypeInfoToLogicalType(getResultType()),
-				new BigIntType()
-		};
-
-		String[] fieldNames = new String[] {
-				"value",
-				"time"
-		};
-
-		return (TypeInformation) new BaseRowTypeInfo(fieldTypes, fieldNames);
-	}
-
-	/**
-	 * Built-in Byte FirstValue aggregate function.
-	 */
-	public static class ByteFirstValueAggFunction extends FirstValueAggFunction<Byte> {
-
-		@Override
-		public TypeInformation<Byte> getResultType() {
-			return Types.BYTE;
-		}
-	}
-
-	/**
-	 * Built-in Short FirstValue aggregate function.
-	 */
-	public static class ShortFirstValueAggFunction extends FirstValueAggFunction<Short> {
-
-		@Override
-		public TypeInformation<Short> getResultType() {
-			return Types.SHORT;
-		}
-	}
-
-	/**
-	 * Built-in Int FirstValue aggregate function.
-	 */
-	public static class IntFirstValueAggFunction extends FirstValueAggFunction<Integer> {
-
-		@Override
-		public TypeInformation<Integer> getResultType() {
-			return Types.INT;
-		}
-	}
-
-	/**
-	 * Built-in Long FirstValue aggregate function.
-	 */
-	public static class LongFirstValueAggFunction extends FirstValueAggFunction<Long> {
-
-		@Override
-		public TypeInformation<Long> getResultType() {
-			return Types.LONG;
-		}
-	}
-
-	/**
-	 * Built-in Float FirstValue aggregate function.
-	 */
-	public static class FloatFirstValueAggFunction extends FirstValueAggFunction<Float> {
-
-		@Override
-		public TypeInformation<Float> getResultType() {
-			return Types.FLOAT;
-		}
-	}
-
-	/**
-	 * Built-in Double FirstValue aggregate function.
-	 */
-	public static class DoubleFirstValueAggFunction extends FirstValueAggFunction<Double> {
-
-		@Override
-		public TypeInformation<Double> getResultType() {
-			return Types.DOUBLE;
-		}
-	}
-
-	/**
-	 * Built-in Boolean FirstValue aggregate function.
-	 */
-	public static class BooleanFirstValueAggFunction extends FirstValueAggFunction<Boolean> {
-
-		@Override
-		public TypeInformation<Boolean> getResultType() {
-			return Types.BOOLEAN;
-		}
-	}
-
-	/**
-	 * Built-in Decimal FirstValue aggregate function.
-	 */
-	public static class DecimalFirstValueAggFunction extends FirstValueAggFunction<Decimal> {
-
-		private DecimalTypeInfo decimalTypeInfo;
-
-		public DecimalFirstValueAggFunction(DecimalTypeInfo decimalTypeInfo) {
-			this.decimalTypeInfo = decimalTypeInfo;
-		}
-
-		public void accumulate(GenericRow acc, Decimal value) {
-			super.accumulate(acc, value);
-		}
-
-		public void accumulate(GenericRow acc, Decimal value, Long order) {
-			super.accumulate(acc, value, order);
-		}
-
-		@Override
-		public TypeInformation<Decimal> getResultType() {
-			return decimalTypeInfo;
-		}
-	}
-
-
-	/**
-	 * Built-in String FirstValue aggregate function.
-	 */
-	public static class StringFirstValueAggFunction extends FirstValueAggFunction<BinaryString> {
-
-		@Override
-		public TypeInformation<BinaryString> getResultType() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value) {
-			if (value != null) {
-				super.accumulate(acc, value.copy());
-			}
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value, Long order) {
-			// just ignore nulls values and orders
-			if (value != null) {
-				super.accumulate(acc, value.copy(), order);
-			}
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
deleted file mode 100644
index 799815a..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/FirstValueWithRetractAggFunction.java
+++ /dev/null
@@ -1,408 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
-import org.apache.flink.api.common.typeutils.base.ByteSerializer;
-import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
-import org.apache.flink.api.common.typeutils.base.FloatSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.ListSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.MapSerializer;
-import org.apache.flink.api.common.typeutils.base.ShortSerializer;
-import org.apache.flink.api.java.typeutils.ListTypeInfo;
-import org.apache.flink.table.api.dataview.MapView;
-import org.apache.flink.table.dataformat.BinaryGeneric;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.dataformat.GenericRow;
-import org.apache.flink.table.dataview.MapViewSerializer;
-import org.apache.flink.table.dataview.MapViewTypeInfo;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalSerializer;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-import org.apache.flink.table.types.logical.BigIntType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.TypeInformationAnyType;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
-
-/**
- * built-in FirstValue with retraction aggregate function.
- */
-public abstract class FirstValueWithRetractAggFunction<T> extends AggregateFunction<T, GenericRow> {
-
-	@Override
-	public GenericRow createAccumulator() {
-		// The accumulator schema:
-		// firstValue: T
-		// fistOrder: Long
-		// valueToOrderMap: BinaryGeneric<MapView<T, List<Long>>>
-		// orderToValueMap: BinaryGeneric<MapView<Long, List<T>>>
-		GenericRow acc = new GenericRow(4);
-		acc.setField(0, null);
-		acc.setField(1, null);
-		acc.setField(2, new BinaryGeneric<>(
-			new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)),
-			getValueToOrderMapViewSerializer()));
-		acc.setField(3, new BinaryGeneric<>(
-			new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())),
-			getOrderToValueMapViewSerializer()));
-		return acc;
-	}
-
-	public void accumulate(GenericRow acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			Long order = System.currentTimeMillis();
-			MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-			List<Long> orderList = valueToOrderMapView.get(v);
-			if (orderList == null) {
-				orderList = new ArrayList<>();
-			}
-			orderList.add(order);
-			valueToOrderMapView.put(v, orderList);
-			accumulate(acc, value, order);
-		}
-	}
-
-	public void accumulate(GenericRow acc, Object value, Long order) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			Long prevOrder = (Long) acc.getField(1);
-			if (prevOrder == null || prevOrder > order) {
-				acc.setField(0, v); // acc.firstValue = v
-				acc.setLong(1, order); // acc.firstOrder = order
-			}
-
-			MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-			List<T> valueList = orderToValueMapView.get(order);
-			if (valueList == null) {
-				valueList = new ArrayList<>();
-			}
-			valueList.add(v);
-			orderToValueMapView.put(order, valueList);
-		}
-	}
-
-	public void retract(GenericRow acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-			List<Long> orderList = valueToOrderMapView.get(v);
-			if (orderList != null && orderList.size() > 0) {
-				Long order = orderList.get(0);
-				orderList.remove(0);
-				if (orderList.isEmpty()) {
-					valueToOrderMapView.remove(v);
-				} else {
-					valueToOrderMapView.put(v, orderList);
-				}
-				retract(acc, value, order);
-			}
-		}
-	}
-
-	public void retract(GenericRow acc, Object value, Long order) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-			List<T> valueList = orderToValueMapView.get(order);
-			if (valueList == null) {
-				return;
-			}
-			int index = valueList.indexOf(v);
-			if (index >= 0) {
-				valueList.remove(index);
-				if (valueList.isEmpty()) {
-					orderToValueMapView.remove(order);
-				} else {
-					orderToValueMapView.put(order, valueList);
-				}
-			}
-			if (v.equals(acc.getField(0))) { // v == acc.firstValue
-				Long startKey = (Long) acc.getField(1);
-				Iterator<Long> iter = orderToValueMapView.keys().iterator();
-				// find the minimal order which is greater than or equal to `startKey`
-				Long nextKey = Long.MAX_VALUE;
-				while (iter.hasNext()) {
-					Long key = iter.next();
-					if (key >= startKey && key < nextKey) {
-						nextKey = key;
-					}
-				}
-
-				if (nextKey != Long.MAX_VALUE) {
-					acc.setField(0, orderToValueMapView.get(nextKey).get(0));
-					acc.setField(1, nextKey);
-				} else {
-					acc.setField(0, null);
-					acc.setField(1, null);
-				}
-			}
-		}
-	}
-
-	public void resetAccumulator(GenericRow acc) {
-		acc.setField(0, null);
-		acc.setField(1, null);
-		MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-		valueToOrderMapView.clear();
-		MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-		orderToValueMapView.clear();
-	}
-
-	@Override
-	public T getValue(GenericRow acc) {
-		return (T) acc.getField(0);
-	}
-
-	protected abstract TypeSerializer<T> createValueSerializer();
-
-	@Override
-	public TypeInformation<GenericRow> getAccumulatorType() {
-		LogicalType[] fieldTypes = new LogicalType[] {
-				fromTypeInfoToLogicalType(getResultType()),
-				new BigIntType(),
-				new TypeInformationAnyType<>(new MapViewTypeInfo<>(getResultType(), new ListTypeInfo<>(Types.LONG), false, false)),
-				new TypeInformationAnyType<>(new MapViewTypeInfo<>(Types.LONG, new ListTypeInfo<>(getResultType()), false, false))
-		};
-
-		String[] fieldNames = new String[] {
-				"firstValue",
-				"firstOrder",
-				"valueToOrderMapView",
-				"orderToValueMapView"
-		};
-
-		return (TypeInformation) new BaseRowTypeInfo(fieldTypes, fieldNames);
-	}
-
-	@SuppressWarnings("unchecked")
-	private MapView<T, List<Long>> getValueToOrderMapViewFromAcc(GenericRow acc) {
-		BinaryGeneric<MapView<T, List<Long>>> binaryGeneric =
-				(BinaryGeneric<MapView<T, List<Long>>>) acc.getField(2);
-		return BinaryGeneric.getJavaObjectFromBinaryGeneric(binaryGeneric, getValueToOrderMapViewSerializer());
-	}
-
-	@SuppressWarnings("unchecked")
-	private MapView<Long, List<T>> getOrderToValueMapViewFromAcc(GenericRow acc) {
-		BinaryGeneric<MapView<Long, List<T>>> binaryGeneric =
-				(BinaryGeneric<MapView<Long, List<T>>>) acc.getField(3);
-		return BinaryGeneric.getJavaObjectFromBinaryGeneric(binaryGeneric, getOrderToValueMapViewSerializer());
-	}
-
-	// MapView<T, List<Long>>
-	private MapViewSerializer<T, List<Long>> getValueToOrderMapViewSerializer() {
-		return new MapViewSerializer<>(
-				new MapSerializer<>(
-						createValueSerializer(),
-						new ListSerializer<>(LongSerializer.INSTANCE)));
-	}
-
-	// MapView<Long, List<T>>
-	private MapViewSerializer<Long, List<T>> getOrderToValueMapViewSerializer() {
-		return new MapViewSerializer<>(
-				new MapSerializer<>(
-						LongSerializer.INSTANCE,
-						new ListSerializer<>(createValueSerializer())));
-	}
-
-	/**
-	 * Built-in Byte FirstValue with retract aggregate function.
-	 */
-	public static class ByteFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Byte> {
-
-		@Override
-		public TypeInformation<Byte> getResultType() {
-			return Types.BYTE;
-		}
-
-		@Override
-		protected TypeSerializer<Byte> createValueSerializer() {
-			return ByteSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Short FirstValue with retract aggregate function.
-	 */
-	public static class ShortFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Short> {
-
-		@Override
-		public TypeInformation<Short> getResultType() {
-			return Types.SHORT;
-		}
-
-		@Override
-		protected TypeSerializer<Short> createValueSerializer() {
-			return ShortSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Int FirstValue with retract aggregate function.
-	 */
-	public static class IntFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Integer> {
-
-		@Override
-		public TypeInformation<Integer> getResultType() {
-			return Types.INT;
-		}
-
-		@Override
-		protected TypeSerializer<Integer> createValueSerializer() {
-			return IntSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Long FirstValue with retract aggregate function.
-	 */
-	public static class LongFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Long> {
-
-		@Override
-		public TypeInformation<Long> getResultType() {
-			return Types.LONG;
-		}
-
-		@Override
-		protected TypeSerializer<Long> createValueSerializer() {
-			return LongSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Float FirstValue with retract aggregate function.
-	 */
-	public static class FloatFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Float> {
-
-		@Override
-		public TypeInformation<Float> getResultType() {
-			return Types.FLOAT;
-		}
-
-		@Override
-		protected TypeSerializer<Float> createValueSerializer() {
-			return FloatSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Double FirstValue with retract aggregate function.
-	 */
-	public static class DoubleFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Double> {
-
-		@Override
-		public TypeInformation<Double> getResultType() {
-			return Types.DOUBLE;
-		}
-
-		@Override
-		protected TypeSerializer<Double> createValueSerializer() {
-			return DoubleSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Boolean FirstValue with retract aggregate function.
-	 */
-	public static class BooleanFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Boolean> {
-
-		@Override
-		public TypeInformation<Boolean> getResultType() {
-			return Types.BOOLEAN;
-		}
-
-		@Override
-		protected TypeSerializer<Boolean> createValueSerializer() {
-			return BooleanSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Decimal FirstValue with retract aggregate function.
-	 */
-	public static class DecimalFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<Decimal> {
-
-		private DecimalTypeInfo decimalTypeInfo;
-
-		public DecimalFirstValueWithRetractAggFunction(DecimalTypeInfo decimalTypeInfo) {
-			this.decimalTypeInfo = decimalTypeInfo;
-		}
-
-		public void accumulate(GenericRow acc, Decimal value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void accumulate(GenericRow acc, Decimal value, Long order) throws Exception {
-			super.accumulate(acc, value, order);
-		}
-
-		@Override
-		public TypeInformation<Decimal> getResultType() {
-			return decimalTypeInfo;
-		}
-
-		@Override
-		protected TypeSerializer<Decimal> createValueSerializer() {
-			return new DecimalSerializer(decimalTypeInfo.precision(), decimalTypeInfo.scale());
-		}
-	}
-
-	/**
-	 * Built-in String FirstValue with retract aggregate function.
-	 */
-	public static class StringFirstValueWithRetractAggFunction extends FirstValueWithRetractAggFunction<BinaryString> {
-
-		@Override
-		public TypeInformation<BinaryString> getResultType() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value) throws Exception {
-			if (value != null) {
-				super.accumulate(acc, value.copy());
-			}
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value, Long order) throws Exception {
-			// just ignore nulls values and orders
-			if (value != null) {
-				super.accumulate(acc, value.copy(), order);
-			}
-		}
-
-		@Override
-		protected TypeSerializer<BinaryString> createValueSerializer() {
-			return BinaryStringSerializer.INSTANCE;
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumAggFunction.java
deleted file mode 100644
index 9735432..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumAggFunction.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.TableException;
-import org.apache.flink.table.calcite.FlinkTypeSystem;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.lessThan;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.ExpressionBuilder.or;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in IncrSum aggregate function,
- * negative number is discarded to ensure the monotonicity.
- */
-public abstract class IncrSumAggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression sum = unresolvedRef("sum");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { sum };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { getResultType() };
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* sum = */ nullOf(getResultType())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* sum = */
-				ifThenElse(or(isNull(operand(0)), lessThan(operand(0), literal(0L))), sum,
-						ifThenElse(isNull(sum), operand(0), plus(sum, operand(0))))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		throw new TableException("This function does not support retraction, Please choose SumWithRetractAggFunction.");
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* sum = */
-				ifThenElse(isNull(mergeOperand(sum)), sum,
-						ifThenElse(isNull(sum), mergeOperand(sum), plus(sum, mergeOperand(sum))))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return sum;
-	}
-
-	/**
-	 * Built-in Int IncrSum aggregate function.
-	 */
-	public static class IntIncrSumAggFunction extends IncrSumAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.INT();
-		}
-	}
-
-	/**
-	 * Built-in Byte IncrSum aggregate function.
-	 */
-	public static class ByteIncrSumAggFunction extends IncrSumAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TINYINT();
-		}
-	}
-
-	/**
-	 * Built-in Short IncrSum aggregate function.
-	 */
-	public static class ShortIncrSumAggFunction extends IncrSumAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.SMALLINT();
-		}
-	}
-
-	/**
-	 * Built-in Long IncrSum aggregate function.
-	 */
-	public static class LongIncrSumAggFunction extends IncrSumAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BIGINT();
-		}
-	}
-
-	/**
-	 * Built-in Float IncrSum aggregate function.
-	 */
-	public static class FloatIncrSumAggFunction extends IncrSumAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.FLOAT();
-		}
-	}
-
-	/**
-	 * Built-in Double IncrSum aggregate function.
-	 */
-	public static class DoubleIncrSumAggFunction extends IncrSumAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-	}
-
-	/**
-	 * Built-in Decimal IncrSum aggregate function.
-	 */
-	public static class DecimalIncrSumAggFunction extends IncrSumAggFunction {
-		private DecimalType decimalType;
-
-		public DecimalIncrSumAggFunction(DecimalType decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		@Override
-		public DataType getResultType() {
-			DecimalType sumType = FlinkTypeSystem.inferAggSumType(decimalType.getScale());
-			return DataTypes.DECIMAL(sumType.getPrecision(), sumType.getScale());
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumWithRetractAggFunction.java
deleted file mode 100644
index a6513be..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/IncrSumWithRetractAggFunction.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.calcite.FlinkTypeSystem;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.equalTo;
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.lessThan;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.ExpressionBuilder.or;
-import static org.apache.flink.table.expressions.ExpressionBuilder.plus;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in IncrSum with retract aggregate function,
- * negative number is discarded to ensure the monotonicity.
- */
-public abstract class IncrSumWithRetractAggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression sum = unresolvedRef("sum");
-	private UnresolvedReferenceExpression count = unresolvedRef("count");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[0];
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] {
-				getResultType(),
-				DataTypes.BIGINT() };
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* sum = */ nullOf(getResultType()),
-				/* count = */ literal(0L)
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* sum = */
-				ifThenElse(or(isNull(operand(0)), lessThan(operand(0), zeroLiteral())), sum,
-						ifThenElse(isNull(sum), operand(0), plus(sum, operand(0)))),
-				/* count = */
-				ifThenElse(or(isNull(operand(0)), lessThan(operand(0), literal(0L))), count,
-						plus(count, literal(1L)))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		return new Expression[] {
-				/* sum = */
-				ifThenElse(or(isNull(operand(0)), lessThan(operand(0), zeroLiteral())), sum,
-						ifThenElse(isNull(sum), minus(zeroLiteral(), operand(0)), minus(sum, operand(0)))),
-				/* count = */
-				ifThenElse(isNull(operand(0)), count, minus(count, literal(1L)))
-		};
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* sum = */
-				ifThenElse(isNull(mergeOperand(sum)), sum,
-						ifThenElse(isNull(sum), mergeOperand(sum), plus(sum, mergeOperand(sum)))),
-				/* count = */
-				plus(count, mergeOperand(count))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return ifThenElse(equalTo(count, literal(0L)), nullOf(getResultType()), sum);
-	}
-
-	protected abstract Expression zeroLiteral();
-
-	/**
-	 * Built-in IncrInt Sum with retract aggregate function.
-	 */
-	public static class IntIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.INT();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal(0);
-		}
-	}
-
-	/**
-	 * Built-in Byte IncrSum with retract aggregate function.
-	 */
-	public static class ByteIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TINYINT();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal((byte) 0);
-		}
-	}
-
-	/**
-	 * Built-in Short IncrSum with retract aggregate function.
-	 */
-	public static class ShortIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.SMALLINT();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal((short) 0);
-		}
-	}
-
-	/**
-	 * Built-in Long IncrSum with retract aggregate function.
-	 */
-	public static class LongIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BIGINT();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal(0L);
-		}
-	}
-
-	/**
-	 * Built-in Float IncrSum with retract aggregate function.
-	 */
-	public static class FloatIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.FLOAT();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal(0F);
-		}
-	}
-
-	/**
-	 * Built-in Double IncrSum with retract aggregate function.
-	 */
-	public static class DoubleIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal(0D);
-		}
-	}
-
-	/**
-	 * Built-in Decimal IncrSum with retract aggregate function.
-	 */
-	public static class DecimalIncrSumWithRetractAggFunction extends IncrSumWithRetractAggFunction {
-		private DecimalType decimalType;
-
-		public DecimalIncrSumWithRetractAggFunction(DecimalType decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		@Override
-		public DataType getResultType() {
-			DecimalType sumType = FlinkTypeSystem.inferAggSumType(decimalType.getScale());
-			return DataTypes.DECIMAL(sumType.getPrecision(), sumType.getScale());
-		}
-
-		@Override
-		protected Expression zeroLiteral() {
-			return literal(0);
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
deleted file mode 100644
index 5846735..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueAggFunction.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.dataformat.GenericRow;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-import org.apache.flink.table.types.logical.BigIntType;
-import org.apache.flink.table.types.logical.LogicalType;
-
-import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
-
-/**
- * built-in LastValue aggregate function.
- */
-public class LastValueAggFunction<T> extends AggregateFunction<T, GenericRow> {
-
-	@Override
-	public boolean isDeterministic() {
-		return false;
-	}
-
-	@Override
-	public GenericRow createAccumulator() {
-		// The accumulator schema:
-		// lastValue: T
-		// lastOrder: Long
-		GenericRow acc = new GenericRow(2);
-		acc.setField(0, null);
-		acc.setLong(1, Long.MIN_VALUE);
-		return acc;
-	}
-
-	public void accumulate(GenericRow acc, Object value) {
-		if (value != null) {
-			acc.setField(0, value);
-		}
-	}
-
-	public void accumulate(GenericRow acc, Object value, Long order) {
-		if (value != null && acc.getLong(1) < order) {
-			acc.setField(0, value);
-			acc.setLong(1, order);
-		}
-	}
-
-	public void resetAccumulator(GenericRow acc) {
-		acc.setField(0, null);
-		acc.setLong(1, Long.MIN_VALUE);
-	}
-
-	@Override
-	public T getValue(GenericRow acc) {
-		return (T) acc.getField(0);
-	}
-
-	@Override
-	public TypeInformation<GenericRow> getAccumulatorType() {
-		LogicalType[] fieldTypes = new LogicalType[] {
-				fromTypeInfoToLogicalType(getResultType()),
-				new BigIntType()
-		};
-
-		String[] fieldNames = new String[] {
-				"value",
-				"time"
-		};
-
-		return (TypeInformation) new BaseRowTypeInfo(fieldTypes, fieldNames);
-	}
-
-	/**
-	 * Built-in Byte LastValue aggregate function.
-	 */
-	public static class ByteLastValueAggFunction extends LastValueAggFunction<Byte> {
-
-		@Override
-		public TypeInformation<Byte> getResultType() {
-			return Types.BYTE;
-		}
-	}
-
-	/**
-	 * Built-in Short LastValue aggregate function.
-	 */
-	public static class ShortLastValueAggFunction extends LastValueAggFunction<Short> {
-
-		@Override
-		public TypeInformation<Short> getResultType() {
-			return Types.SHORT;
-		}
-	}
-
-	/**
-	 * Built-in Int LastValue aggregate function.
-	 */
-	public static class IntLastValueAggFunction extends LastValueAggFunction<Integer> {
-
-		@Override
-		public TypeInformation<Integer> getResultType() {
-			return Types.INT;
-		}
-	}
-
-	/**
-	 * Built-in Long LastValue aggregate function.
-	 */
-	public static class LongLastValueAggFunction extends LastValueAggFunction<Long> {
-
-		@Override
-		public TypeInformation<Long> getResultType() {
-			return Types.LONG;
-		}
-	}
-
-	/**
-	 * Built-in Float LastValue aggregate function.
-	 */
-	public static class FloatLastValueAggFunction extends LastValueAggFunction<Float> {
-
-		@Override
-		public TypeInformation<Float> getResultType() {
-			return Types.FLOAT;
-		}
-	}
-
-	/**
-	 * Built-in Double LastValue aggregate function.
-	 */
-	public static class DoubleLastValueAggFunction extends LastValueAggFunction<Double> {
-
-		@Override
-		public TypeInformation<Double> getResultType() {
-			return Types.DOUBLE;
-		}
-	}
-
-	/**
-	 * Built-in Boolean LastValue aggregate function.
-	 */
-	public static class BooleanLastValueAggFunction extends LastValueAggFunction<Boolean> {
-
-		@Override
-		public TypeInformation<Boolean> getResultType() {
-			return Types.BOOLEAN;
-		}
-	}
-
-	/**
-	 * Built-in Decimal LastValue aggregate function.
-	 */
-	public static class DecimalLastValueAggFunction extends LastValueAggFunction<Decimal> {
-
-		private DecimalTypeInfo decimalTypeInfo;
-
-		public DecimalLastValueAggFunction(DecimalTypeInfo decimalTypeInfo) {
-			this.decimalTypeInfo = decimalTypeInfo;
-		}
-
-		public void accumulate(GenericRow acc, Decimal value) {
-			super.accumulate(acc, value);
-		}
-
-		public void accumulate(GenericRow acc, Decimal value, Long order) {
-			super.accumulate(acc, value, order);
-		}
-
-		@Override
-		public TypeInformation<Decimal> getResultType() {
-			return decimalTypeInfo;
-		}
-	}
-
-
-	/**
-	 * Built-in String LastValue aggregate function.
-	 */
-	public static class StringLastValueAggFunction extends LastValueAggFunction<BinaryString> {
-
-		@Override
-		public TypeInformation<BinaryString> getResultType() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value) {
-			if (value != null) {
-				super.accumulate(acc, value.copy());
-			}
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value, Long order) {
-			// just ignore nulls values and orders
-			if (value != null) {
-				super.accumulate(acc, value.copy(), order);
-			}
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
deleted file mode 100644
index 6455844..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LastValueWithRetractAggFunction.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
-import org.apache.flink.api.common.typeutils.base.ByteSerializer;
-import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
-import org.apache.flink.api.common.typeutils.base.FloatSerializer;
-import org.apache.flink.api.common.typeutils.base.IntSerializer;
-import org.apache.flink.api.common.typeutils.base.ListSerializer;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.common.typeutils.base.MapSerializer;
-import org.apache.flink.api.common.typeutils.base.ShortSerializer;
-import org.apache.flink.api.java.typeutils.ListTypeInfo;
-import org.apache.flink.table.api.dataview.MapView;
-import org.apache.flink.table.dataformat.BinaryGeneric;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.dataformat.GenericRow;
-import org.apache.flink.table.dataview.MapViewSerializer;
-import org.apache.flink.table.dataview.MapViewTypeInfo;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BaseRowTypeInfo;
-import org.apache.flink.table.runtime.typeutils.BinaryStringSerializer;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalSerializer;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-import org.apache.flink.table.types.logical.BigIntType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.TypeInformationAnyType;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import static org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType;
-
-/**
- * built-in LastValue with retraction aggregate function.
- */
-public abstract class LastValueWithRetractAggFunction<T> extends AggregateFunction<T, GenericRow> {
-
-	@Override
-	public GenericRow createAccumulator() {
-		// The accumulator schema:
-		// lastValue: T
-		// lastOrder: Long
-		// valueToOrderMap: BinaryGeneric<MapView<T, List<Long>>>
-		// orderToValueMap: BinaryGeneric<MapView<Long, List<T>>>
-		GenericRow acc = new GenericRow(4);
-		acc.setField(0, null);
-		acc.setField(1, null);
-		acc.setField(2, new BinaryGeneric<>(
-				new MapView<>(getResultType(), new ListTypeInfo<>(Types.LONG)), getValueToOrderMapViewSerializer()));
-		acc.setField(3, new BinaryGeneric<>(
-				new MapView<>(Types.LONG, new ListTypeInfo<>(getResultType())), getOrderToValueMapViewSerializer()));
-		return acc;
-	}
-
-	public void accumulate(GenericRow acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			Long order = System.currentTimeMillis();
-			MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-			List<Long> orderList = valueToOrderMapView.get(v);
-			if (orderList == null) {
-				orderList = new ArrayList<>();
-			}
-			orderList.add(order);
-			valueToOrderMapView.put(v, orderList);
-			accumulate(acc, value, order);
-		}
-	}
-
-	public void accumulate(GenericRow acc, Object value, Long order) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			Long prevOrder = (Long) acc.getField(1);
-			if (prevOrder == null || prevOrder <= order) {
-				acc.setField(0, v); // acc.lastValue = v
-				acc.setLong(1, order); // acc.lastOrder = order
-			}
-
-			MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-			List<T> valueList = orderToValueMapView.get(order);
-			if (valueList == null) {
-				valueList = new ArrayList<>();
-			}
-			valueList.add(v);
-			orderToValueMapView.put(order, valueList);
-		}
-	}
-
-	public void retract(GenericRow acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-			List<Long> orderList = valueToOrderMapView.get(v);
-			if (orderList != null && orderList.size() > 0) {
-				Long order = orderList.get(0);
-				orderList.remove(0);
-				if (orderList.isEmpty()) {
-					valueToOrderMapView.remove(v);
-				} else {
-					valueToOrderMapView.put(v, orderList);
-				}
-				retract(acc, value, order);
-			}
-		}
-	}
-
-	public void retract(GenericRow acc, Object value, Long order) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-			MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-			List<T> valueList = orderToValueMapView.get(order);
-			if (valueList == null) {
-				return;
-			}
-			int index = valueList.indexOf(v);
-			if (index >= 0) {
-				valueList.remove(index);
-				if (valueList.isEmpty()) {
-					orderToValueMapView.remove(order);
-				} else {
-					orderToValueMapView.put(order, valueList);
-				}
-			}
-			if (v.equals(acc.getField(0))) { // v == acc.firstValue
-				Long startKey = (Long) acc.getField(1);
-				Iterator<Long> iter = orderToValueMapView.keys().iterator();
-				// find the maximal order which is less than or equal to `startKey`
-				Long nextKey = Long.MIN_VALUE;
-				while (iter.hasNext()) {
-					Long key = iter.next();
-					if (key <= startKey && key > nextKey) {
-						nextKey = key;
-					}
-				}
-
-				if (nextKey != Long.MIN_VALUE) {
-					List<T> values = orderToValueMapView.get(nextKey);
-					acc.setField(0, values.get(values.size() - 1));
-					acc.setField(1, nextKey);
-				} else {
-					acc.setField(0, null);
-					acc.setField(1, null);
-				}
-			}
-		}
-	}
-
-	public void resetAccumulator(GenericRow acc) {
-		acc.setField(0, null);
-		acc.setField(1, null);
-		MapView<T, List<Long>> valueToOrderMapView = getValueToOrderMapViewFromAcc(acc);
-		valueToOrderMapView.clear();
-		MapView<Long, List<T>> orderToValueMapView = getOrderToValueMapViewFromAcc(acc);
-		orderToValueMapView.clear();
-	}
-
-	@Override
-	public T getValue(GenericRow acc) {
-		return (T) acc.getField(0);
-	}
-
-	protected abstract TypeSerializer<T> createValueSerializer();
-
-	@Override
-	public TypeInformation<GenericRow> getAccumulatorType() {
-		LogicalType[] fieldTypes = new LogicalType[] {
-				fromTypeInfoToLogicalType(getResultType()),
-				new BigIntType(),
-				new TypeInformationAnyType<>(new MapViewTypeInfo<>(getResultType(), new ListTypeInfo<>(Types.LONG), false, false)),
-				new TypeInformationAnyType<>(new MapViewTypeInfo<>(Types.LONG, new ListTypeInfo<>(getResultType()), false, false))
-		};
-
-		String[] fieldNames = new String[] {
-				"lastValue",
-				"lastOrder",
-				"valueToOrderMapView",
-				"orderToValueMapView"
-		};
-
-		return (TypeInformation) new BaseRowTypeInfo(fieldTypes, fieldNames);
-	}
-
-	@SuppressWarnings("unchecked")
-	private MapView<T, List<Long>> getValueToOrderMapViewFromAcc(GenericRow acc) {
-		BinaryGeneric<MapView<T, List<Long>>> binaryGeneric =
-				(BinaryGeneric<MapView<T, List<Long>>>) acc.getField(2);
-		return BinaryGeneric.getJavaObjectFromBinaryGeneric(binaryGeneric, getValueToOrderMapViewSerializer());
-	}
-
-	@SuppressWarnings("unchecked")
-	private MapView<Long, List<T>> getOrderToValueMapViewFromAcc(GenericRow acc) {
-		BinaryGeneric<MapView<Long, List<T>>> binaryGeneric =
-				(BinaryGeneric<MapView<Long, List<T>>>) acc.getField(3);
-		return BinaryGeneric.getJavaObjectFromBinaryGeneric(binaryGeneric, getOrderToValueMapViewSerializer());
-	}
-
-	// MapView<T, List<Long>>
-	private MapViewSerializer<T, List<Long>> getValueToOrderMapViewSerializer() {
-		return new MapViewSerializer<>(
-				new MapSerializer<>(
-						createValueSerializer(),
-						new ListSerializer<>(LongSerializer.INSTANCE)));
-	}
-
-	// MapView<Long, List<T>>
-	private MapViewSerializer<Long, List<T>> getOrderToValueMapViewSerializer() {
-		return new MapViewSerializer<>(
-				new MapSerializer<>(
-						LongSerializer.INSTANCE,
-						new ListSerializer<>(createValueSerializer())));
-	}
-
-	/**
-	 * Built-in Byte LastValue with retract aggregate function.
-	 */
-	public static class ByteLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Byte> {
-
-		@Override
-		public TypeInformation<Byte> getResultType() {
-			return Types.BYTE;
-		}
-
-		@Override
-		protected TypeSerializer<Byte> createValueSerializer() {
-			return ByteSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Short LastValue with retract aggregate function.
-	 */
-	public static class ShortLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Short> {
-
-		@Override
-		public TypeInformation<Short> getResultType() {
-			return Types.SHORT;
-		}
-
-		@Override
-		protected TypeSerializer<Short> createValueSerializer() {
-			return ShortSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Int LastValue with retract aggregate function.
-	 */
-	public static class IntLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Integer> {
-
-		@Override
-		public TypeInformation<Integer> getResultType() {
-			return Types.INT;
-		}
-
-		@Override
-		protected TypeSerializer<Integer> createValueSerializer() {
-			return IntSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Long LastValue with retract aggregate function.
-	 */
-	public static class LongLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Long> {
-
-		@Override
-		public TypeInformation<Long> getResultType() {
-			return Types.LONG;
-		}
-
-		@Override
-		protected TypeSerializer<Long> createValueSerializer() {
-			return LongSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Float LastValue with retract aggregate function.
-	 */
-	public static class FloatLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Float> {
-
-		@Override
-		public TypeInformation<Float> getResultType() {
-			return Types.FLOAT;
-		}
-
-		@Override
-		protected TypeSerializer<Float> createValueSerializer() {
-			return FloatSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Double LastValue with retract aggregate function.
-	 */
-	public static class DoubleLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Double> {
-
-		@Override
-		public TypeInformation<Double> getResultType() {
-			return Types.DOUBLE;
-		}
-
-		@Override
-		protected TypeSerializer<Double> createValueSerializer() {
-			return DoubleSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Boolean LastValue with retract aggregate function.
-	 */
-	public static class BooleanLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Boolean> {
-
-		@Override
-		public TypeInformation<Boolean> getResultType() {
-			return Types.BOOLEAN;
-		}
-
-		@Override
-		protected TypeSerializer<Boolean> createValueSerializer() {
-			return BooleanSerializer.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Decimal LastValue with retract aggregate function.
-	 */
-	public static class DecimalLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<Decimal> {
-
-		private DecimalTypeInfo decimalTypeInfo;
-
-		public DecimalLastValueWithRetractAggFunction(DecimalTypeInfo decimalTypeInfo) {
-			this.decimalTypeInfo = decimalTypeInfo;
-		}
-
-		public void accumulate(GenericRow acc, Decimal value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void accumulate(GenericRow acc, Decimal value, Long order) throws Exception {
-			super.accumulate(acc, value, order);
-		}
-
-		@Override
-		public TypeInformation<Decimal> getResultType() {
-			return decimalTypeInfo;
-		}
-
-		@Override
-		protected TypeSerializer<Decimal> createValueSerializer() {
-			return new DecimalSerializer(decimalTypeInfo.precision(), decimalTypeInfo.scale());
-		}
-	}
-
-	/**
-	 * Built-in String LastValue with retract aggregate function.
-	 */
-	public static class StringLastValueWithRetractAggFunction extends LastValueWithRetractAggFunction<BinaryString> {
-
-		@Override
-		public TypeInformation<BinaryString> getResultType() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value) throws Exception {
-			if (value != null) {
-				super.accumulate(acc, value.copy());
-			}
-		}
-
-		public void accumulate(GenericRow acc, BinaryString value, Long order) throws Exception {
-			// just ignore nulls values and orders
-			if (value != null) {
-				super.accumulate(acc, value.copy(), order);
-			}
-		}
-
-		@Override
-		protected TypeSerializer<BinaryString> createValueSerializer() {
-			return BinaryStringSerializer.INSTANCE;
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
deleted file mode 100644
index 6e9c276..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/LeadLagAggFunction.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.runtime.operators.over.frame.OffsetOverFrame;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.TimeType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.cast;
-import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
-import static org.apache.flink.table.expressions.ExpressionBuilder.typeLiteral;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * LEAD and LAG aggregate functions return the value of given expression evaluated at given offset.
- * The functions only are used by over window.
- *
- * <p>LAG(input, offset, default) - Returns the value of `input` at the `offset`th row
- * before the current row in the window. The default value of `offset` is 1 and the default
- * value of `default` is null. If the value of `input` at the `offset`th row is null,
- * null is returned. If there is no such offset row (e.g., when the offset is 1, the first
- * row of the window does not have any previous row), `default` is returned.
- *
- * <p>LEAD(input, offset, default) - Returns the value of `input` at the `offset`th row
- * after the current row in the window. The default value of `offset` is 1 and the default
- * value of `default` is null. If the value of `input` at the `offset`th row is null,
- * null is returned. If there is no such an offset row (e.g., when the offset is 1, the last
- * row of the window does not have any subsequent row), `default` is returned.
- *
- * <p>These two aggregate functions are special, and only are used by over window. So here the
- * concrete implementation is closely related to {@link OffsetOverFrame}.
- */
-public abstract class LeadLagAggFunction extends DeclarativeAggregateFunction {
-
-	private int operandCount;
-
-	//If the length of function's args is 3, then the function has the default value.
-	private boolean existDefaultValue;
-
-	private UnresolvedReferenceExpression value = unresolvedRef("leadlag");
-
-	public LeadLagAggFunction(int operandCount) {
-		this.operandCount = operandCount;
-		existDefaultValue = operandCount == 3;
-	}
-
-	@Override
-	public int operandCount() {
-		return operandCount;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] {value};
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] {getResultType()};
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {literal(null, getResultType())};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {operand(0)};
-	}
-
-	// TODO hack, use the current input reset the buffer value.
-	@Override
-	public Expression[] retractExpressions() {
-		return new Expression[] {existDefaultValue ? cast(operand(2),
-				typeLiteral(getResultType())) : literal(null, getResultType())};
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[0];
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return value;
-	}
-
-	/**
-	 * IntLeadLagAggFunction.
-	 */
-	public static class IntLeadLagAggFunction extends LeadLagAggFunction {
-
-		public IntLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.INT();
-		}
-	}
-
-	/**
-	 * ByteLeadLagAggFunction.
-	 */
-	public static class ByteLeadLagAggFunction extends LeadLagAggFunction {
-
-		public ByteLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TINYINT();
-		}
-	}
-
-	/**
-	 * ShortLeadLagAggFunction.
-	 */
-	public static class ShortLeadLagAggFunction extends LeadLagAggFunction {
-
-		public ShortLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.SMALLINT();
-		}
-	}
-
-	/**
-	 * LongLeadLagAggFunction.
-	 */
-	public static class LongLeadLagAggFunction extends LeadLagAggFunction {
-
-		public LongLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BIGINT();
-		}
-	}
-
-	/**
-	 * FloatLeadLagAggFunction.
-	 */
-	public static class FloatLeadLagAggFunction extends LeadLagAggFunction {
-
-		public FloatLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.FLOAT();
-		}
-	}
-
-	/**
-	 * DoubleLeadLagAggFunction.
-	 */
-	public static class DoubleLeadLagAggFunction extends LeadLagAggFunction {
-
-		public DoubleLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-	}
-
-	/**
-	 * BooleanLeadLagAggFunction.
-	 */
-	public static class BooleanLeadLagAggFunction extends LeadLagAggFunction {
-
-		public BooleanLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BOOLEAN();
-		}
-	}
-
-	/**
-	 * DecimalLeadLagAggFunction.
-	 */
-	public static class DecimalLeadLagAggFunction extends LeadLagAggFunction {
-
-		private final DecimalType decimalType;
-
-		public DecimalLeadLagAggFunction(int operandCount, DecimalType decimalType) {
-			super(operandCount);
-			this.decimalType = decimalType;
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale());
-		}
-	}
-
-	/**
-	 * StringLeadLagAggFunction.
-	 */
-	public static class StringLeadLagAggFunction extends LeadLagAggFunction {
-
-		public StringLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.STRING();
-		}
-	}
-
-	/**
-	 * DateLeadLagAggFunction.
-	 */
-	public static class DateLeadLagAggFunction extends LeadLagAggFunction {
-
-		public DateLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DATE();
-		}
-	}
-
-	/**
-	 * TimeLeadLagAggFunction.
-	 */
-	public static class TimeLeadLagAggFunction extends LeadLagAggFunction {
-
-		public TimeLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIME(TimeType.DEFAULT_PRECISION);
-		}
-	}
-
-	/**
-	 * TimestampLeadLagAggFunction.
-	 */
-	public static class TimestampLeadLagAggFunction extends LeadLagAggFunction {
-
-		public TimestampLeadLagAggFunction(int operandCount) {
-			super(operandCount);
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIMESTAMP(3);
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.java
deleted file mode 100644
index 91ffa59..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxAggFunction.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.TimeType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.greaterThan;
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in max aggregate function.
- */
-public abstract class MaxAggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression max = unresolvedRef("max");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { max };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { getResultType() };
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* max = */ nullOf(getResultType())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* max = */
-				ifThenElse(isNull(operand(0)), max,
-						ifThenElse(isNull(max), operand(0),
-								ifThenElse(greaterThan(operand(0), max), operand(0), max)))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		// TODO FLINK-12295, ignore exception now
-//		throw new TableException("This function does not support retraction, Please choose MaxWithRetractAggFunction.");
-		return new Expression[0];
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* max = */
-				ifThenElse(isNull(mergeOperand(max)), max,
-						ifThenElse(isNull(max), mergeOperand(max),
-								ifThenElse(greaterThan(mergeOperand(max), max), mergeOperand(max), max)))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return max;
-	}
-
-	/**
-	 * Built-in Int Max aggregate function.
-	 */
-	public static class IntMaxAggFunction extends MaxAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.INT();
-		}
-	}
-
-	/**
-	 * Built-in Byte Max aggregate function.
-	 */
-	public static class ByteMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TINYINT();
-		}
-	}
-
-	/**
-	 * Built-in Short Max aggregate function.
-	 */
-	public static class ShortMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.SMALLINT();
-		}
-	}
-
-	/**
-	 * Built-in Long Max aggregate function.
-	 */
-	public static class LongMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BIGINT();
-		}
-	}
-
-	/**
-	 * Built-in Float Max aggregate function.
-	 */
-	public static class FloatMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.FLOAT();
-		}
-	}
-
-	/**
-	 * Built-in Double Max aggregate function.
-	 */
-	public static class DoubleMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-	}
-
-	/**
-	 * Built-in Decimal Max aggregate function.
-	 */
-	public static class DecimalMaxAggFunction extends MaxAggFunction {
-		private DecimalType decimalType;
-
-		public DecimalMaxAggFunction(DecimalType decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale());
-		}
-	}
-
-	/**
-	 * Built-in Boolean Max aggregate function.
-	 */
-	public static class BooleanMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BOOLEAN();
-		}
-	}
-
-	/**
-	 * Built-in String Max aggregate function.
-	 */
-	public static class StringMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.STRING();
-		}
-	}
-
-	/**
-	 * Built-in Date Max aggregate function.
-	 */
-	public static class DateMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DATE();
-		}
-	}
-
-	/**
-	 * Built-in Time Max aggregate function.
-	 */
-	public static class TimeMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIME(TimeType.DEFAULT_PRECISION);
-		}
-	}
-
-	/**
-	 * Built-in Timestamp Max aggregate function.
-	 */
-	public static class TimestampMaxAggFunction extends MaxAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIMESTAMP(3);
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
deleted file mode 100644
index 888cbfe..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MaxWithRetractAggFunction.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.PojoField;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.table.api.dataview.MapView;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * built-in Max with retraction aggregate function.
- */
-public abstract class MaxWithRetractAggFunction<T extends Comparable>
-		extends AggregateFunction<T, MaxWithRetractAggFunction.MaxWithRetractAccumulator<T>> {
-
-	private static final long serialVersionUID = -5860934997657147836L;
-
-	/** The initial accumulator for Max with retraction aggregate function. */
-	public static class MaxWithRetractAccumulator<T> {
-		public T max;
-		public Long mapSize;
-		public MapView<T, Long> map;
-	}
-
-	@Override
-	public MaxWithRetractAccumulator<T> createAccumulator() {
-		MaxWithRetractAccumulator<T> acc = new MaxWithRetractAccumulator<>();
-		acc.max = null; // max
-		acc.mapSize = 0L;
-		// store the count for each value
-		acc.map = new MapView<>(getValueTypeInfo(), BasicTypeInfo.LONG_TYPE_INFO);
-		return acc;
-	}
-
-	public void accumulate(MaxWithRetractAccumulator<T> acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-
-			if (acc.mapSize == 0L || acc.max.compareTo(v) < 0) {
-				acc.max = v;
-			}
-
-			Long count = acc.map.get(v);
-			if (count == null) {
-				count = 0L;
-			}
-			count += 1L;
-			if (count == 0) {
-				// remove it when count is increased from -1 to 0
-				acc.map.remove(v);
-			} else {
-				// store it when count is NOT zero
-				acc.map.put(v, count);
-			}
-			if (count == 1L) {
-				// previous count is zero, this is the first time to see the key
-				acc.mapSize += 1;
-			}
-		}
-	}
-
-	public void retract(MaxWithRetractAccumulator<T> acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-
-			Long count = acc.map.get(v);
-			if (count == null) {
-				count = 0L;
-			}
-			count -= 1;
-			if (count == 0) {
-				// remove it when count is decreased from 1 to 0
-				acc.map.remove(v);
-				acc.mapSize -= 1L;
-
-				//if the total count is 0, we could just simply set the f0(max) to the initial value
-				if (acc.mapSize == 0) {
-					acc.max = null;
-					return;
-				}
-				//if v is the current max value, we have to iterate the map to find the 2nd biggest
-				// value to replace v as the max value
-				if (v.equals(acc.max)) {
-					updateMax(acc);
-				}
-			} else {
-				// store it when count is NOT zero
-				acc.map.put(v, count);
-				// we do not take negative number account into mapSize
-			}
-		}
-	}
-
-	private void updateMax(MaxWithRetractAccumulator<T> acc) throws Exception {
-		boolean hasMax = false;
-		for (T key : acc.map.keys()) {
-			if (!hasMax || acc.max.compareTo(key) < 0) {
-				acc.max = key;
-				hasMax = true;
-			}
-		}
-		// The behavior of deleting expired data in the state backend is uncertain.
-		// so `mapSize` data may exist, while `map` data may have been deleted
-		// when both of them are expired.
-		if (!hasMax) {
-			acc.mapSize = 0L;
-			// we should also override max value, because it may have an old value.
-			acc.max = null;
-		}
-	}
-
-	public void merge(MaxWithRetractAccumulator<T> acc, Iterable<MaxWithRetractAccumulator<T>> its) throws Exception {
-		boolean needUpdateMax = false;
-		for (MaxWithRetractAccumulator<T> a : its) {
-			// set max element
-			if (acc.mapSize == 0 || (a.mapSize > 0 && a.max != null && acc.max.compareTo(a.max) < 0)) {
-				acc.max = a.max;
-			}
-			// merge the count for each key
-			for (Map.Entry entry : a.map.entries()) {
-				T key = (T) entry.getKey();
-				Long otherCount = (Long) entry.getValue(); // non-null
-				Long thisCount = acc.map.get(key);
-				if (thisCount == null) {
-					thisCount = 0L;
-				}
-				long mergedCount = otherCount + thisCount;
-				if (mergedCount == 0) {
-					// remove it when count is increased from -1 to 0
-					acc.map.remove(key);
-					if (thisCount > 0) {
-						// origin is > 0, and retract to 0
-						acc.mapSize -= 1;
-						if (key.equals(acc.max)) {
-							needUpdateMax = true;
-						}
-					}
-				} else if (mergedCount < 0) {
-					acc.map.put(key, mergedCount);
-					if (thisCount > 0) {
-						// origin is > 0, and retract to < 0
-						acc.mapSize -= 1;
-						if (key.equals(acc.max)) {
-							needUpdateMax = true;
-						}
-					}
-				} else { // mergedCount > 0
-					acc.map.put(key, mergedCount);
-					if (thisCount <= 0) {
-						// origin is <= 0, and accumulate to > 0
-						acc.mapSize += 1;
-					}
-				}
-			}
-		}
-		if (needUpdateMax) {
-			updateMax(acc);
-		}
-	}
-
-	public void resetAccumulator(MaxWithRetractAccumulator<T> acc) {
-		acc.max = null;
-		acc.mapSize = 0L;
-		acc.map.clear();
-	}
-
-	@Override
-	public T getValue(MaxWithRetractAccumulator<T> acc) {
-		if (acc.mapSize > 0) {
-			return acc.max;
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public TypeInformation<MaxWithRetractAccumulator<T>> getAccumulatorType() {
-		PojoTypeInfo pojoType = (PojoTypeInfo) TypeExtractor.createTypeInfo(MaxWithRetractAccumulator.class);
-		List<PojoField> pojoFields = new ArrayList<>();
-		for (int i = 0; i < pojoType.getTotalFields(); i++) {
-			PojoField field = pojoType.getPojoFieldAt(i);
-			if (field.getField().getName().equals("max")) {
-				pojoFields.add(new PojoField(field.getField(), getValueTypeInfo()));
-			} else {
-				pojoFields.add(field);
-			}
-		}
-		//noinspection unchecked
-		return new PojoTypeInfo(pojoType.getTypeClass(), pojoFields);
-	}
-
-	@Override
-	public TypeInformation<T> getResultType() {
-		return getValueTypeInfo();
-	}
-
-	protected abstract TypeInformation<T> getValueTypeInfo();
-
-	/**
-	 * Built-in Byte Max with retraction aggregate function.
-	 */
-	public static class ByteMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Byte> {
-
-		private static final long serialVersionUID = 7383980948808353819L;
-
-		@Override
-		protected TypeInformation<Byte> getValueTypeInfo() {
-			return BasicTypeInfo.BYTE_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Short Max with retraction aggregate function.
-	 */
-	public static class ShortMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Short> {
-
-		private static final long serialVersionUID = 7579072678911328694L;
-
-		@Override
-		protected TypeInformation<Short> getValueTypeInfo() {
-			return BasicTypeInfo.SHORT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Int Max with retraction aggregate function.
-	 */
-	public static class IntMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Integer> {
-
-		private static final long serialVersionUID = 3833976566544263072L;
-
-		@Override
-		protected TypeInformation<Integer> getValueTypeInfo() {
-			return BasicTypeInfo.INT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Long Max with retraction aggregate function.
-	 */
-	public static class LongMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Long> {
-
-		private static final long serialVersionUID = 8585384188523017375L;
-
-		@Override
-		protected TypeInformation<Long> getValueTypeInfo() {
-			return BasicTypeInfo.LONG_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Float Max with retraction aggregate function.
-	 */
-	public static class FloatMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Float> {
-
-		private static final long serialVersionUID = -1433882434794024584L;
-
-		@Override
-		protected TypeInformation<Float> getValueTypeInfo() {
-			return BasicTypeInfo.FLOAT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Double Max with retraction aggregate function.
-	 */
-	public static class DoubleMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Double> {
-
-		private static final long serialVersionUID = -1525221057708740308L;
-
-		@Override
-		protected TypeInformation<Double> getValueTypeInfo() {
-			return BasicTypeInfo.DOUBLE_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Boolean Max with retraction aggregate function.
-	 */
-	public static class BooleanMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Boolean> {
-
-		private static final long serialVersionUID = -8408715018822625309L;
-
-		@Override
-		protected TypeInformation<Boolean> getValueTypeInfo() {
-			return BasicTypeInfo.BOOLEAN_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Big Decimal Max with retraction aggregate function.
-	 */
-	public static class DecimalMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Decimal> {
-		private static final long serialVersionUID = 5301860581297042635L;
-		private DecimalTypeInfo decimalType;
-
-		public DecimalMaxWithRetractAggFunction(DecimalTypeInfo decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		public void accumulate(MaxWithRetractAccumulator<Decimal> acc, Decimal value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void retract(MaxWithRetractAccumulator<Decimal> acc, Decimal value) throws Exception {
-			super.retract(acc, value);
-		}
-
-		@Override
-		protected TypeInformation<Decimal> getValueTypeInfo() {
-			return decimalType;
-		}
-	}
-
-	/**
-	 * Built-in String Max with retraction aggregate function.
-	 */
-	public static class StringMaxWithRetractAggFunction extends MaxWithRetractAggFunction<BinaryString> {
-
-		private static final long serialVersionUID = 787528574867514796L;
-
-		public void accumulate(MaxWithRetractAccumulator<BinaryString> acc, BinaryString value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void retract(MaxWithRetractAccumulator<BinaryString> acc, BinaryString value) throws Exception {
-			super.retract(acc, value);
-		}
-
-		@Override
-		protected TypeInformation<BinaryString> getValueTypeInfo() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Timestamp Max with retraction aggregate function.
-	 */
-	public static class TimestampMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Timestamp> {
-
-		private static final long serialVersionUID = -7096481949093142944L;
-
-		@Override
-		protected TypeInformation<Timestamp> getValueTypeInfo() {
-			return Types.SQL_TIMESTAMP;
-		}
-	}
-
-	/**
-	 * Built-in Date Max with retraction aggregate function.
-	 */
-	public static class DateMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Date> {
-
-		private static final long serialVersionUID = 7452698503075473023L;
-
-		@Override
-		protected TypeInformation<Date> getValueTypeInfo() {
-			return Types.SQL_DATE;
-		}
-	}
-
-	/**
-	 * Built-in Time Max with retraction aggregate function.
-	 */
-	public static class TimeMaxWithRetractAggFunction extends MaxWithRetractAggFunction<Time> {
-
-		private static final long serialVersionUID = 3578216747876121493L;
-
-		@Override
-		protected TypeInformation<Time> getValueTypeInfo() {
-			return Types.SQL_TIME;
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinAggFunction.java
deleted file mode 100644
index 7e5f27b..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinAggFunction.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.expressions.Expression;
-import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.TimeType;
-
-import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
-import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
-import static org.apache.flink.table.expressions.ExpressionBuilder.lessThan;
-import static org.apache.flink.table.expressions.ExpressionBuilder.nullOf;
-import static org.apache.flink.table.expressions.utils.ApiExpressionUtils.unresolvedRef;
-
-/**
- * built-in min aggregate function.
- */
-public abstract class MinAggFunction extends DeclarativeAggregateFunction {
-	private UnresolvedReferenceExpression min = unresolvedRef("min");
-
-	@Override
-	public int operandCount() {
-		return 1;
-	}
-
-	@Override
-	public UnresolvedReferenceExpression[] aggBufferAttributes() {
-		return new UnresolvedReferenceExpression[] { min };
-	}
-
-	@Override
-	public DataType[] getAggBufferTypes() {
-		return new DataType[] { getResultType() };
-	}
-
-	@Override
-	public Expression[] initialValuesExpressions() {
-		return new Expression[] {
-				/* min = */ nullOf(getResultType())
-		};
-	}
-
-	@Override
-	public Expression[] accumulateExpressions() {
-		return new Expression[] {
-				/* min = */
-				ifThenElse(isNull(operand(0)), min,
-						ifThenElse(isNull(min), operand(0),
-								ifThenElse(lessThan(operand(0), min), operand(0), min)))
-		};
-	}
-
-	@Override
-	public Expression[] retractExpressions() {
-		// TODO FLINK-12295, ignore exception now
-//		throw new TableException("This function does not support retraction, Please choose MinWithRetractAggFunction.");
-		return new Expression[0];
-	}
-
-	@Override
-	public Expression[] mergeExpressions() {
-		return new Expression[] {
-				/* min = */
-				ifThenElse(isNull(mergeOperand(min)), min,
-						ifThenElse(isNull(min), mergeOperand(min),
-								ifThenElse(lessThan(mergeOperand(min), min), mergeOperand(min), min)))
-		};
-	}
-
-	@Override
-	public Expression getValueExpression() {
-		return min;
-	}
-
-	/**
-	 * Built-in Int Min aggregate function.
-	 */
-	public static class IntMinAggFunction extends MinAggFunction {
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.INT();
-		}
-	}
-
-	/**
-	 * Built-in Byte Min aggregate function.
-	 */
-	public static class ByteMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TINYINT();
-		}
-	}
-
-	/**
-	 * Built-in Short Min aggregate function.
-	 */
-	public static class ShortMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.SMALLINT();
-		}
-	}
-
-	/**
-	 * Built-in Long Min aggregate function.
-	 */
-	public static class LongMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BIGINT();
-		}
-	}
-
-	/**
-	 * Built-in Float Min aggregate function.
-	 */
-	public static class FloatMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.FLOAT();
-		}
-	}
-
-	/**
-	 * Built-in Double Min aggregate function.
-	 */
-	public static class DoubleMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DOUBLE();
-		}
-	}
-
-	/**
-	 * Built-in Decimal Min aggregate function.
-	 */
-	public static class DecimalMinAggFunction extends MinAggFunction {
-		private DecimalType decimalType;
-
-		public DecimalMinAggFunction(DecimalType decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale());
-		}
-	}
-
-	/**
-	 * Built-in Boolean Min aggregate function.
-	 */
-	public static class BooleanMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.BOOLEAN();
-		}
-	}
-
-	/**
-	 * Built-in String Min aggregate function.
-	 */
-	public static class StringMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.STRING();
-		}
-	}
-
-	/**
-	 * Built-in Date Min aggregate function.
-	 */
-	public static class DateMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.DATE();
-		}
-	}
-
-	/**
-	 * Built-in Time Min aggregate function.
-	 */
-	public static class TimeMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIME(TimeType.DEFAULT_PRECISION);
-		}
-	}
-
-	/**
-	 * Built-in Timestamp Min aggregate function.
-	 */
-	public static class TimestampMinAggFunction extends MinAggFunction {
-		@Override
-		public DataType getResultType() {
-			return DataTypes.TIMESTAMP(3);
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
deleted file mode 100644
index f3c21bf..0000000
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/MinWithRetractAggFunction.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.functions.aggfunctions;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.PojoField;
-import org.apache.flink.api.java.typeutils.PojoTypeInfo;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.table.api.dataview.MapView;
-import org.apache.flink.table.dataformat.BinaryString;
-import org.apache.flink.table.dataformat.Decimal;
-import org.apache.flink.table.functions.AggregateFunction;
-import org.apache.flink.table.runtime.typeutils.BinaryStringTypeInfo;
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo;
-
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * built-in Min with retraction aggregate function.
- */
-public abstract class MinWithRetractAggFunction<T extends Comparable>
-		extends AggregateFunction<T, MinWithRetractAggFunction.MinWithRetractAccumulator<T>> {
-
-	private static final long serialVersionUID = 4253774292802374843L;
-
-	/** The initial accumulator for Min with retraction aggregate function. */
-	public static class MinWithRetractAccumulator<T> {
-		public T min;
-		public Long mapSize;
-		public MapView<T, Long> map;
-	}
-
-	@Override
-	public MinWithRetractAccumulator<T> createAccumulator() {
-		MinWithRetractAccumulator<T> acc = new MinWithRetractAccumulator<>();
-		acc.min = null; // min
-		acc.mapSize = 0L;
-		// store the count for each value
-		acc.map = new MapView<>(getValueTypeInfo(), BasicTypeInfo.LONG_TYPE_INFO);
-		return acc;
-	}
-
-	public void accumulate(MinWithRetractAccumulator<T> acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-
-			if (acc.mapSize == 0L || acc.min.compareTo(v) > 0) {
-				acc.min = v;
-			}
-
-			Long count = acc.map.get(v);
-			if (count == null) {
-				count = 0L;
-			}
-			count += 1L;
-			if (count == 0) {
-				// remove it when count is increased from -1 to 0
-				acc.map.remove(v);
-			} else {
-				// store it when count is NOT zero
-				acc.map.put(v, count);
-			}
-			if (count == 1L) {
-				// previous count is zero, this is the first time to see the key
-				acc.mapSize += 1;
-			}
-		}
-	}
-
-	public void retract(MinWithRetractAccumulator<T> acc, Object value) throws Exception {
-		if (value != null) {
-			T v = (T) value;
-
-			Long count = acc.map.get(v);
-			if (count == null) {
-				count = 0L;
-			}
-			count -= 1;
-			if (count == 0) {
-				// remove it when count is decreased from 1 to 0
-				acc.map.remove(v);
-				acc.mapSize -= 1L;
-
-				//if the total count is 0, we could just simply set the f0(min) to the initial value
-				if (acc.mapSize == 0) {
-					acc.min = null;
-					return;
-				}
-				//if v is the current min value, we have to iterate the map to find the 2nd biggest
-				// value to replace v as the min value
-				if (v.equals(acc.min)) {
-					updateMin(acc);
-				}
-			} else {
-				// store it when count is NOT zero
-				acc.map.put(v, count);
-				// we do not take negative number account into mapSize
-			}
-		}
-	}
-
-	private void updateMin(MinWithRetractAccumulator<T> acc) throws Exception {
-		boolean hasMin = false;
-		for (T key : acc.map.keys()) {
-			if (!hasMin || acc.min.compareTo(key) > 0) {
-				acc.min = key;
-				hasMin = true;
-			}
-		}
-		// The behavior of deleting expired data in the state backend is uncertain.
-		// so `mapSize` data may exist, while `map` data may have been deleted
-		// when both of them are expired.
-		if (!hasMin) {
-			acc.mapSize = 0L;
-			// we should also override min value, because it may have an old value.
-			acc.min = null;
-		}
-	}
-
-	public void merge(MinWithRetractAccumulator<T> acc, Iterable<MinWithRetractAccumulator<T>> its) throws Exception {
-		boolean needUpdateMin = false;
-		for (MinWithRetractAccumulator<T> a : its) {
-			// set min element
-			if (acc.mapSize == 0 || (a.mapSize > 0 && a.min != null && acc.min.compareTo(a.min) > 0)) {
-				acc.min = a.min;
-			}
-			// merge the count for each key
-			for (Map.Entry entry : a.map.entries()) {
-				T key = (T) entry.getKey();
-				Long otherCount = (Long) entry.getValue(); // non-null
-				Long thisCount = acc.map.get(key);
-				if (thisCount == null) {
-					thisCount = 0L;
-				}
-				long mergedCount = otherCount + thisCount;
-				if (mergedCount == 0) {
-					// remove it when count is increased from -1 to 0
-					acc.map.remove(key);
-					if (thisCount > 0) {
-						// origin is > 0, and retract to 0
-						acc.mapSize -= 1;
-						if (key.equals(acc.min)) {
-							needUpdateMin = true;
-						}
-					}
-				} else if (mergedCount < 0) {
-					acc.map.put(key, mergedCount);
-					if (thisCount > 0) {
-						// origin is > 0, and retract to < 0
-						acc.mapSize -= 1;
-						if (key.equals(acc.min)) {
-							needUpdateMin = true;
-						}
-					}
-				} else { // mergedCount > 0
-					acc.map.put(key, mergedCount);
-					if (thisCount <= 0) {
-						// origin is <= 0, and accumulate to > 0
-						acc.mapSize += 1;
-					}
-				}
-			}
-		}
-		if (needUpdateMin) {
-			updateMin(acc);
-		}
-	}
-
-	public void resetAccumulator(MinWithRetractAccumulator<T> acc) {
-		acc.min = null;
-		acc.mapSize = 0L;
-		acc.map.clear();
-	}
-
-	@Override
-	public T getValue(MinWithRetractAccumulator<T> acc) {
-		if (acc.mapSize > 0) {
-			return acc.min;
-		} else {
-			return null;
-		}
-	}
-
-	@Override
-	public TypeInformation<MinWithRetractAccumulator<T>> getAccumulatorType() {
-		PojoTypeInfo pojoType = (PojoTypeInfo) TypeExtractor.createTypeInfo(MinWithRetractAccumulator.class);
-		List<PojoField> pojoFields = new ArrayList<>();
-		for (int i = 0; i < pojoType.getTotalFields(); i++) {
-			PojoField field = pojoType.getPojoFieldAt(i);
-			if (field.getField().getName().equals("min")) {
-				pojoFields.add(new PojoField(field.getField(), getValueTypeInfo()));
-			} else {
-				pojoFields.add(field);
-			}
-		}
-		//noinspection unchecked
-		return new PojoTypeInfo(pojoType.getTypeClass(), pojoFields);
-	}
-
-	@Override
-	public TypeInformation<T> getResultType() {
-		return getValueTypeInfo();
-	}
-
-	protected abstract TypeInformation<T> getValueTypeInfo();
-
-	/**
-	 * Built-in Byte Min with retraction aggregate function.
-	 */
-	public static class ByteMinWithRetractAggFunction extends MinWithRetractAggFunction<Byte> {
-
-		private static final long serialVersionUID = 3170462557144510063L;
-
-		@Override
-		protected TypeInformation<Byte> getValueTypeInfo() {
-			return BasicTypeInfo.BYTE_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Short Min with retraction aggregate function.
-	 */
-	public static class ShortMinWithRetractAggFunction extends MinWithRetractAggFunction<Short> {
-
-		private static final long serialVersionUID = -4877567451203730974L;
-
-		@Override
-		protected TypeInformation<Short> getValueTypeInfo() {
-			return BasicTypeInfo.SHORT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Int Min with retraction aggregate function.
-	 */
-	public static class IntMinWithRetractAggFunction extends MinWithRetractAggFunction<Integer> {
-
-		private static final long serialVersionUID = -3187801696860321834L;
-
-		@Override
-		protected TypeInformation<Integer> getValueTypeInfo() {
-			return BasicTypeInfo.INT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Long Min with retraction aggregate function.
-	 */
-	public static class LongMinWithRetractAggFunction extends MinWithRetractAggFunction<Long> {
-
-		private static final long serialVersionUID = -3224670103852172282L;
-
-		@Override
-		protected TypeInformation<Long> getValueTypeInfo() {
-			return BasicTypeInfo.LONG_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Float Min with retraction aggregate function.
-	 */
-	public static class FloatMinWithRetractAggFunction extends MinWithRetractAggFunction<Float> {
-
-		private static final long serialVersionUID = 6683867851550125554L;
-
-		@Override
-		protected TypeInformation<Float> getValueTypeInfo() {
-			return BasicTypeInfo.FLOAT_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Double Min with retraction aggregate function.
-	 */
-	public static class DoubleMinWithRetractAggFunction extends MinWithRetractAggFunction<Double> {
-
-		private static final long serialVersionUID = -9107897474595423074L;
-
-		@Override
-		protected TypeInformation<Double> getValueTypeInfo() {
-			return BasicTypeInfo.DOUBLE_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Boolean Min with retraction aggregate function.
-	 */
-	public static class BooleanMinWithRetractAggFunction extends MinWithRetractAggFunction<Boolean> {
-
-		private static final long serialVersionUID = -4667566512148979776L;
-
-		@Override
-		protected TypeInformation<Boolean> getValueTypeInfo() {
-			return BasicTypeInfo.BOOLEAN_TYPE_INFO;
-		}
-	}
-
-	/**
-	 * Built-in Big Decimal Min with retraction aggregate function.
-	 */
-	public static class DecimalMinWithRetractAggFunction extends MinWithRetractAggFunction<Decimal> {
-		private static final long serialVersionUID = -7984016112363017960L;
-		private DecimalTypeInfo decimalType;
-
-		public DecimalMinWithRetractAggFunction(DecimalTypeInfo decimalType) {
-			this.decimalType = decimalType;
-		}
-
-		public void accumulate(MinWithRetractAccumulator<Decimal> acc, Decimal value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void retract(MinWithRetractAccumulator<Decimal> acc, Decimal value) throws Exception {
-			super.retract(acc, value);
-		}
-
-		@Override
-		protected TypeInformation<Decimal> getValueTypeInfo() {
-			return decimalType;
-		}
-	}
-
-	/**
-	 * Built-in String Min with retraction aggregate function.
-	 */
-	public static class StringMinWithRetractAggFunction extends MinWithRetractAggFunction<BinaryString> {
-
-		private static final long serialVersionUID = -6402993104400269468L;
-
-		public void accumulate(MinWithRetractAccumulator<BinaryString> acc, BinaryString value) throws Exception {
-			super.accumulate(acc, value);
-		}
-
-		public void retract(MinWithRetractAccumulator<BinaryString> acc, BinaryString value) throws Exception {
-			super.retract(acc, value);
-		}
-
-		@Override
-		protected TypeInformation<BinaryString> getValueTypeInfo() {
-			return BinaryStringTypeInfo.INSTANCE;
-		}
-	}
-
-	/**
-	 * Built-in Timestamp Min with retraction aggregate function.
-	 */
-	public static class TimestampMinWithRetractAggFunction extends MinWithRetractAggFunction<Timestamp> {
-
-		private static final long serialVersionUID = -7494198823345305907L;
-
-		@Override
-		protected TypeInformation<Timestamp> getValueTypeInfo() {
-			return Types.SQL_TIMESTAMP;
-		}
-	}
-
-	/**
-	 * Built-in Date Min with retraction aggregate function.
-	 */
-	public static class DateMinWithRetractAggFunction extends MinWithRetractAggFunction<Date> {
-
-		private static final long serialVersionUID = 604406649989470870L;
-
-		@Override
-		protected TypeInformation<Date> getValueTypeInfo() {
-			return Types.SQL_DATE;
-		}
-	}
-
-	/**
-	 * Built-in Time Min with retraction aggregate function.
-	 */
-	public static class TimeMinWithRetractAggFunction extends MinWithRetractAggFunction<Time> {
-
-		private static final long serialVersionUID = -6908371577415696291L;
-
-		@Override
-		protected TypeInformation<Time> getValueTypeInfo() {
-			return Types.SQL_TIME;
-		}
-	}
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/RankAggFunction.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/functions/aggfunctions/RankAggFunction.java
deleted file mode 100644
... 431543 lines suppressed ...