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


##########
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 . For this case:
   ```
   String ddl =  "CREATE TABLE MyTable1 (\n"
                           + "  metadata_0 int METADATA VIRTUAL,\n"
                           + "  a0 int,\n"
                           + "  a1 int,\n"
                           + "  a2 int,\n"
                           + "  ts STRING,\n "
                           + "  rowtime as TO_TIMESTAMP(`ts`),\n"
                           + "  WATERMARK FOR rowtime AS rowtime - INTERVAL '1' 
SECOND\n"
                           + ") WITH ( .....
                           + ")";
   ```
   which `METADATA` column defined as first column.  
   ```
   
DynamicSourceUtils.createProducedType(contextResolvedTable.getResolvedSchema(), 
tableSource)
   ``` 
   will get different schema order with
   ```
   contextResolvedTable.getResolvedSchema()
                   .toSourceRowDataType()
                   .getLogicalType()
   ```
   
   IMO, I think it's better to use the origin order than convert the order to 
`PHYSICAL COLUMNS + METADATA COLUMNS`, but this need @twalthr to confirm it.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to