twalthr commented on a change in pull request #16285:
URL: https://github.com/apache/flink/pull/16285#discussion_r663818201



##########
File path: docs/content.zh/docs/dev/table/common.md
##########
@@ -324,15 +324,18 @@ Such tables can either be created using the Table API 
directly, or by switching
 
 ```java
 // Using table descriptors
-tableEnv.createTemporaryTable("Source", TableDescriptor.forConnector("datagen")
+final TableDescriptor sourceDescriptor = 
TableDescriptor.forConnector("datagen")
     .schema(Schema.newBuilder()
-      .column("f0", DataTypes.STRING())
-      .build())
-    .option(DataGenOptions.ROWS_PER_SECOND, 100)
+    .column("f0", DataTypes.STRING())
     .build())
+    .option(DataGenOptions.ROWS_PER_SECOND, 100)
+    .build();
+
+    tableEnv.createTable("SourceTableA", sourceDescriptor);

Review comment:
       wrong indention for all lines that start with `tableEnv`

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
##########
@@ -548,4 +549,24 @@ class CalcITCase extends StreamingTestBase {
     TestBaseUtils.compareResultAsText(result, "42")
   }
 
+  @Test

Review comment:
       I agree. Having one ITCase for descriptors was ok in the last PR. But we 
should not add more. We can extend 
`org.apache.flink.table.api.TableEnvironmentTest` and check what has been put 
into the in-memory catalog. 

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##########
@@ -499,6 +499,21 @@ public void createTemporaryTable(String path, 
TableDescriptor descriptor) {
         catalogManager.createTemporaryTable(catalogTable, tableIdentifier, 
false);
     }
 
+    @Override
+    public void createTable(String path, TableDescriptor descriptor) {
+        final ObjectIdentifier tableIdentifier =

Review comment:
       add a null check similar to `createTemporaryTable`




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