wuchong commented on a change in pull request #10224: 
[FLINK-14716][table-planner-blink] Cooperate computed column with push down 
rules
URL: https://github.com/apache/flink/pull/10224#discussion_r347261294
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/FlinkCalciteCatalogReader.java
 ##########
 @@ -62,17 +83,197 @@ public FlinkCalciteCatalogReader(
                if (originRelOptTable == null) {
                        return null;
                } else {
-                       // Wrap FlinkTable as FlinkRelOptTable to use in query 
optimization.
-                       FlinkTable table = 
originRelOptTable.unwrap(FlinkTable.class);
+                       // Wrap as linkPreparingTableBase to use in query 
optimization.
+                       CatalogSchemaTable table = 
originRelOptTable.unwrap(CatalogSchemaTable.class);
                        if (table != null) {
-                               return FlinkRelOptTable.create(
-                                       originRelOptTable.getRelOptSchema(),
-                                       originRelOptTable.getRowType(),
+                               return 
toPreparingTable(originRelOptTable.getRelOptSchema(),
                                        originRelOptTable.getQualifiedName(),
+                                       originRelOptTable.getRowType(),
                                        table);
                        } else {
                                return originRelOptTable;
                        }
                }
        }
+
+       /**
+        * Translate this {@link CatalogSchemaTable} into Flink source table.
+        */
+       public static FlinkPreparingTableBase toPreparingTable(RelOptSchema 
relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       CatalogSchemaTable table) {
+               if (table.isTemporary()) {
+                       return convertTemporaryTable(relOptSchema,
+                               names,
+                               rowType,
+                               table.getObjectIdentifier(),
+                               table.getCatalogTable(),
+                               table.getStatistic(),
+                               table.isStreamingMode());
+               } else {
+                       return convertPermanentTable(relOptSchema,
+                               names,
+                               rowType,
+                               table.getObjectIdentifier(),
+                               table.getCatalogTable(),
+                               table.getStatistic(),
+                               
table.getCatalog().getTableFactory().orElse(null),
+                               table.isStreamingMode());
+               }
+       }
+
+       private static FlinkPreparingTableBase convertPermanentTable(
+                       RelOptSchema relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       ObjectIdentifier objectIdentifier,
+                       CatalogBaseTable table,
+                       FlinkStatistic statistic,
+                       @Nullable TableFactory tableFactory,
+                       boolean isStreamingMode) {
+               if (table instanceof QueryOperationCatalogView) {
+                       return convertQueryOperationView(relOptSchema,
+                               names,
+                               rowType,
+                               (QueryOperationCatalogView) table);
+               } else if (table instanceof ConnectorCatalogTable) {
+                       ConnectorCatalogTable<?, ?> connectorTable = 
(ConnectorCatalogTable<?, ?>) table;
+                       if ((connectorTable).getTableSource().isPresent()) {
+                               return convertSourceTable(relOptSchema,
+                                       names,
+                                       rowType,
+                                       connectorTable,
+                                       statistic,
+                                       isStreamingMode);
+                       } else {
+                               throw new ValidationException("Cannot convert a 
connector table " +
+                                       "without source.");
+                       }
+               } else if (table instanceof CatalogTable) {
+                       CatalogTable catalogTable = (CatalogTable) table;
+                       return convertCatalogTable(relOptSchema,
+                               names,
+                               rowType,
+                               objectIdentifier.toObjectPath(),
+                               catalogTable,
+                               tableFactory,
+                               statistic,
+                               isStreamingMode);
+               } else {
+                       throw new ValidationException("Unsupported table type: 
" + table);
+               }
+       }
+
+       private static FlinkPreparingTableBase convertTemporaryTable(
+                       RelOptSchema relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       ObjectIdentifier objectIdentifier,
+                       CatalogBaseTable table,
+                       FlinkStatistic statistic,
+                       boolean isStreamingMode) {
+               if (table instanceof QueryOperationCatalogView) {
+                       return convertQueryOperationView(relOptSchema,
+                               names,
+                               rowType,
+                               (QueryOperationCatalogView) table);
+               } else if (table instanceof ConnectorCatalogTable) {
+                       ConnectorCatalogTable<?, ?> connectorTable = 
(ConnectorCatalogTable<?, ?>) table;
+                       if ((connectorTable).getTableSource().isPresent()) {
+                               return convertSourceTable(relOptSchema,
+                                       names,
+                                       rowType,
+                                       connectorTable,
+                                       statistic,
+                                       isStreamingMode);
+                       } else {
+                               throw new ValidationException("Cannot convert a 
connector table " +
+                                       "without source.");
+                       }
+               } else if (table instanceof CatalogTable) {
+                       return convertCatalogTable(relOptSchema,
+                               names,
+                               rowType,
+                               objectIdentifier.toObjectPath(),
+                               (CatalogTable) table,
+                               null,
+                               statistic,
+                               isStreamingMode);
+               } else {
+                       throw new ValidationException("Unsupported table type: 
" + table);
+               }
+       }
+
+       private static FlinkPreparingTableBase convertQueryOperationView(
+                       RelOptSchema relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       QueryOperationCatalogView view) {
+               return QueryOperationCatalogViewTable.create(relOptSchema, 
names, rowType, view);
+       }
+
+       private static FlinkPreparingTableBase convertSourceTable(
+                       RelOptSchema relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       ConnectorCatalogTable<?, ?> table,
+                       FlinkStatistic statistic,
+                       boolean isStreamingMode) {
+               TableSource<?> tableSource = table.getTableSource().get();
+               if (!(tableSource instanceof StreamTableSource ||
+                       tableSource instanceof LookupableTableSource)) {
+                       throw new ValidationException(
+                               "Only StreamTableSource and 
LookupableTableSource can be used in Blink planner.");
+               }
+               if (!isStreamingMode && tableSource instanceof 
StreamTableSource &&
+                       !((StreamTableSource<?>) tableSource).isBounded()) {
+                       throw new ValidationException("Only bounded 
StreamTableSource can be used in batch mode.");
+               }
+
+               return new TableSourceTable<>(
+                       relOptSchema,
+                       names,
+                       rowType,
+                       tableSource,
+                       isStreamingMode,
+                       statistic,
+                       table);
+       }
+
+       private static FlinkPreparingTableBase convertCatalogTable(
+                       RelOptSchema relOptSchema,
+                       List<String> names,
+                       RelDataType rowType,
+                       ObjectPath tablePath,
+                       CatalogTable table,
+                       @Nullable TableFactory tableFactory,
+                       FlinkStatistic statistic,
+                       boolean isStreamingMode) {
+               TableSource<?> tableSource;
+               if (tableFactory != null) {
+                       if (tableFactory instanceof TableSourceFactory) {
+                               tableSource = ((TableSourceFactory) 
tableFactory).createTableSource(tablePath, table);
+                       } else {
+                               throw new TableException(
+                                       "Cannot query a sink-only table. 
TableFactory provided by catalog must implement TableSourceFactory");
+                       }
+               } else {
+                       tableSource = 
TableFactoryUtil.findAndCreateTableSource(table);
+               }
+
+               if (!(tableSource instanceof StreamTableSource)) {
+                       throw new TableException("Catalog tables support only 
StreamTableSource and InputFormatTableSource");
+               }
+
+               return new TableSourceTable<>(
+                       relOptSchema,
+                       names,
+                       rowType,
+                       tableSource,
+                       !((StreamTableSource<?>) tableSource).isBounded(),
 
 Review comment:
   Yes. I know it's from the old code. But I think the old code is wrong. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to