swuferhong commented on code in PR #23412:
URL: https://github.com/apache/flink/pull/23412#discussion_r1325238554


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java:
##########
@@ -95,29 +92,19 @@ private DynamicTableSource getTableSource(FlinkContext 
context, FlinkTypeFactory
                     FactoryUtil.createDynamicTableSource(
                             factory,
                             contextResolvedTable.getIdentifier(),
-                            resolvedCatalogTable,
+                            contextResolvedTable.getResolvedTable(),
                             loadOptionsFromCatalogTable(contextResolvedTable, 
context),
                             context.getTableConfig(),
                             context.getClassLoader(),
                             contextResolvedTable.isTemporary());
-            // validate DynamicSource and apply Metadata
-            DynamicSourceUtils.prepareDynamicSource(
-                    contextResolvedTable.getIdentifier().toString(),
-                    resolvedCatalogTable,
-                    tableSource,
-                    false,
-                    context.getTableConfig().getConfiguration());
 
             if (sourceAbilities != null) {
-                //  Note: use DynamicSourceUtils.createProducedType to produce 
the type info so that
-                //  keep consistent with sql2Rel phase which also called the 
method producing
-                //  deterministic format (PHYSICAL COLUMNS + METADATA COLUMNS) 
when converts a given
-                //  DynamicTableSource to a RelNode.
-                // TODO should do a refactor(e.g., add serialized input type 
info into each
-                //  SourceAbilitySpec so as to avoid this implicit logic 
dependency)
                 RowType newProducedType =
-                        DynamicSourceUtils.createProducedType(
-                                contextResolvedTable.getResolvedSchema(), 
tableSource);
+                        (RowType)
+                                contextResolvedTable
+                                        .getResolvedSchema()
+                                        .toSourceRowDataType()
+                                        .getLogicalType();

Review Comment:
   Hi, @dawidwys, thanks for your PR. I have a small question that why we 
change the spec column index from `PHYSICAL COLUMNS + METADATA COLUMNS` to the 
original user-defined schema order here ? If we want to change this order, more 
code that uses source specs need to be modified, such as 
`ScanReuser.reuseDuplicatedScan`, cc @twalthr .



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java:
##########
@@ -95,29 +92,19 @@ private DynamicTableSource getTableSource(FlinkContext 
context, FlinkTypeFactory
                     FactoryUtil.createDynamicTableSource(
                             factory,
                             contextResolvedTable.getIdentifier(),
-                            resolvedCatalogTable,
+                            contextResolvedTable.getResolvedTable(),
                             loadOptionsFromCatalogTable(contextResolvedTable, 
context),
                             context.getTableConfig(),
                             context.getClassLoader(),
                             contextResolvedTable.isTemporary());
-            // validate DynamicSource and apply Metadata
-            DynamicSourceUtils.prepareDynamicSource(
-                    contextResolvedTable.getIdentifier().toString(),
-                    resolvedCatalogTable,
-                    tableSource,
-                    false,
-                    context.getTableConfig().getConfiguration());
 
             if (sourceAbilities != null) {
-                //  Note: use DynamicSourceUtils.createProducedType to produce 
the type info so that
-                //  keep consistent with sql2Rel phase which also called the 
method producing
-                //  deterministic format (PHYSICAL COLUMNS + METADATA COLUMNS) 
when converts a given
-                //  DynamicTableSource to a RelNode.
-                // TODO should do a refactor(e.g., add serialized input type 
info into each
-                //  SourceAbilitySpec so as to avoid this implicit logic 
dependency)
                 RowType newProducedType =
-                        DynamicSourceUtils.createProducedType(
-                                contextResolvedTable.getResolvedSchema(), 
tableSource);
+                        (RowType)
+                                contextResolvedTable
+                                        .getResolvedSchema()
+                                        .toSourceRowDataType()
+                                        .getLogicalType();

Review Comment:
   cc @twalthr 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to