KurtYoung commented on a change in pull request #11917:
URL: https://github.com/apache/flink/pull/11917#discussion_r415605040



##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
##########
@@ -649,16 +649,32 @@ public void sqlUpdate(String stmt) {
        private TableResult executeOperation(Operation operation) {
                if (operation instanceof CreateTableOperation) {
                        CreateTableOperation createTableOperation = 
(CreateTableOperation) operation;
-                       catalogManager.createTable(
-                                       createTableOperation.getCatalogTable(),
-                                       
createTableOperation.getTableIdentifier(),
-                                       
createTableOperation.isIgnoreIfExists());
+                       if (createTableOperation.isTemporary()) {
+                               catalogManager.createTemporaryTable(
+                                               
createTableOperation.getCatalogTable(),
+                                               
createTableOperation.getTableIdentifier(),
+                                               
createTableOperation.isIgnoreIfExists());
+                       } else {
+                               catalogManager.createTable(
+                                               
createTableOperation.getCatalogTable(),
+                                               
createTableOperation.getTableIdentifier(),
+                                               
createTableOperation.isIgnoreIfExists());
+                       }
                        return TableResultImpl.TABLE_RESULT_OK;
                } else if (operation instanceof DropTableOperation) {
                        DropTableOperation dropTableOperation = 
(DropTableOperation) operation;
-                       catalogManager.dropTable(
-                                       dropTableOperation.getTableIdentifier(),
-                                       dropTableOperation.isIfExists());
+                       if (dropTableOperation.isTemporary()) {
+                               boolean dropped = 
catalogManager.dropTemporaryTable(dropTableOperation.getTableIdentifier());

Review comment:
       I would suggest to extend `catalogManager.dropTemporaryTable` to take 
care of the logic about `ifExists`, be consistent with 
`catalogManager.dropTable`

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
##########
@@ -137,6 +137,28 @@ class TableEnvironmentTest {
       
.tableExists(ObjectPath.fromString(s"${tableEnv.getCurrentDatabase}.tbl1")))
   }
 
+  @Test
+  def testExecuteSqlWithCreateDropTemporaryTable(): Unit = {

Review comment:
       Testing only the simplest "create + drop" will be insufficient. At least 
we have to cover some illegal cases, adding some cases when temporary & normal 
table co-exists.  

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
##########
@@ -137,6 +137,28 @@ class TableEnvironmentTest {
       
.tableExists(ObjectPath.fromString(s"${tableEnv.getCurrentDatabase}.tbl1")))
   }
 
+  @Test
+  def testExecuteSqlWithCreateDropTemporaryTable(): Unit = {

Review comment:
       BTW, will "show tables" command also show temporary tables?




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


Reply via email to