danny0405 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_r347774404
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java
 ##########
 @@ -80,150 +69,49 @@ public Table getTable(String tableName) {
                return catalogManager.getTable(identifier)
                        .map(result -> {
                                CatalogBaseTable table = result.getTable();
-                               if (result.isTemporary()) {
-                                       return convertTemporaryTable(new 
ObjectPath(databaseName, tableName), table);
-                               } else {
-                                       return convertPermanentTable(
-                                               identifier.toObjectPath(),
-                                               table,
-                                               
catalogManager.getCatalog(catalogName)
-                                                       
.flatMap(Catalog::getTableFactory)
-                                                       .orElse(null)
-                                       );
-                               }
+                               Catalog catalog = 
catalogManager.getCatalog(catalogName).get();
+                               FlinkStatistic statistic = 
getStatistic(result.isTemporary(),
+                                       catalog, table, identifier);
+                               return new CatalogSchemaTable(identifier,
+                                       table,
+                                       statistic,
+                                       catalog.getTableFactory().orElse(null),
+                                       isStreamingMode,
+                                       result.isTemporary());
                        })
                        .orElse(null);
        }
 
-       private Table convertPermanentTable(
-                       ObjectPath tablePath,
-                       CatalogBaseTable table,
-                       @Nullable TableFactory tableFactory) {
-               if (table instanceof QueryOperationCatalogView) {
-                       return 
convertQueryOperationView((QueryOperationCatalogView) table);
-               } else if (table instanceof ConnectorCatalogTable) {
-                       ConnectorCatalogTable<?, ?> connectorTable = 
(ConnectorCatalogTable<?, ?>) table;
-                       if ((connectorTable).getTableSource().isPresent()) {
-                               TableStats tableStats = 
extractTableStats(connectorTable, tablePath);
-                               return convertSourceTable(connectorTable, 
tableStats);
-                       } else {
-                               return convertSinkTable(connectorTable);
-                       }
-               } else if (table instanceof CatalogTable) {
-                       CatalogTable catalogTable = (CatalogTable) table;
-                       TableStats tableStats = extractTableStats(catalogTable, 
tablePath);
-                       return convertCatalogTable(tablePath, catalogTable, 
tableFactory, tableStats);
-               } else {
-                       throw new TableException("Unsupported table type: " + 
table);
-               }
-       }
-
-       private Table convertTemporaryTable(
-                       ObjectPath tablePath,
-                       CatalogBaseTable table) {
-               if (table instanceof QueryOperationCatalogView) {
-                       return 
convertQueryOperationView((QueryOperationCatalogView) table);
-               } else if (table instanceof ConnectorCatalogTable) {
-                       ConnectorCatalogTable<?, ?> connectorTable = 
(ConnectorCatalogTable<?, ?>) table;
-                       if ((connectorTable).getTableSource().isPresent()) {
-                               return convertSourceTable(connectorTable, 
TableStats.UNKNOWN);
-                       } else {
-                               return convertSinkTable(connectorTable);
-                       }
-               } else if (table instanceof CatalogTable) {
-                       return convertCatalogTable(tablePath, (CatalogTable) 
table, null, TableStats.UNKNOWN);
-               } else {
-                       throw new TableException("Unsupported table type: " + 
table);
+       private static FlinkStatistic getStatistic(boolean isTemporary, Catalog 
catalog,
+                       CatalogBaseTable catalogBaseTable, ObjectIdentifier 
tableIdentifier) {
+               if (isTemporary || catalogBaseTable instanceof 
QueryOperationCatalogView) {
+                       return FlinkStatistic.UNKNOWN();
                }
-       }
-
-       private Table convertQueryOperationView(QueryOperationCatalogView 
table) {
-               return QueryOperationCatalogViewTable.createCalciteTable(table);
-       }
-
-       private Table convertSinkTable(ConnectorCatalogTable<?, ?> table) {
-               Optional<TableSinkTable> tableSinkTable = table.getTableSink()
-                       .map(tableSink -> new TableSinkTable<>(
-                               tableSink,
-                               FlinkStatistic.UNKNOWN()));
-               if (tableSinkTable.isPresent()) {
-                       return tableSinkTable.get();
+               if (catalogBaseTable instanceof CatalogTable) {
+                       return FlinkStatistic.builder()
+                               .tableStats(extractTableStats(catalog, 
tableIdentifier))
+                               .build();
                } else {
-                       throw new TableException("Cannot convert a connector 
table " +
-                               "without either source or sink.");
-               }
-       }
-
-       private Table convertSourceTable(
-                       ConnectorCatalogTable<?, ?> table,
-                       TableStats tableStats) {
-               TableSource<?> tableSource = table.getTableSource().get();
-               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.");
+                       throw new TableException("Unsupported table type: " + 
catalogBaseTable);
 
 Review comment:
   Okey, let me change it to `FlinkStatistic.UNKNOWN`, i'm curious if it would 
be cause some wried bug when we get a `FlinkStatistic.UNKNOWN` just because the 
table type is not right.

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


With regards,
Apache Git Services

Reply via email to