godfreyhe commented on a change in pull request #11869:
URL: https://github.com/apache/flink/pull/11869#discussion_r414343381



##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java
##########
@@ -415,6 +415,37 @@ public CatalogBaseTable getTable() {
                        });
        }
 
+       /**
+        * Returns an array of names of all views(both temporary and permanent) 
registered in
+        * the namespace of the current catalog and database.
+        *
+        * @return names of all registered views
+        */
+       public Set<String> listViews() {
+               return listViews(getCurrentCatalog(), getCurrentDatabase());
+       }
+
+       /**
+        * Returns an array of names of all views(both temporary and permanent) 
registered in
+        * the namespace of the current catalog and database.
+        *
+        * @return names of registered views
+        */
+       public Set<String> listViews(String catalogName, String databaseName) {
+               Catalog currentCatalog = catalogs.get(getCurrentCatalog());
+
+               try {
+                       return Stream.concat(
+                               
currentCatalog.listViews(getCurrentDatabase()).stream(),
+                               listTemporaryTablesInternal(catalogName, 
databaseName)
+                                       .filter(e -> e.getValue() instanceof 
CatalogView)
+                                       .map(e -> e.getKey().getObjectName())
+                       ).collect(Collectors.toSet());

Review comment:
       add a method `listTemporaryViews(String catalogName, String 
databaseName)`, `listTemporaryViews()` method and here can use this method

##########
File path: 
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/batch/BatchTableEnvironmentTest.scala
##########
@@ -257,6 +257,64 @@ class BatchTableEnvironmentTest extends TableTestBase {
     util.tableEnv.executeSql("select * from MyTable")
   }
 
+  @Test
+  def testExecuteSqlWithCreateDropView(): Unit = {
+    val util = batchTestUtil()
+
+    val createTableStmt =
+      """
+        |CREATE TABLE tbl1 (
+        |  a bigint,
+        |  b int,
+        |  c varchar
+        |) with (
+        |  'connector' = 'COLLECTION',
+        |  'is-bounded' = 'true'
+        |)
+      """.stripMargin
+    val tableResult1 = util.tableEnv.executeSql(createTableStmt)
+    assertEquals(ResultKind.SUCCESS, tableResult1.getResultKind)
+    assertTrue(util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get()
+      
.tableExists(ObjectPath.fromString(s"${util.tableEnv.getCurrentDatabase}.tbl1")))
+
+    val tableResult2 = util.tableEnv.executeSql("CREATE VIEW view1 AS SELECT * 
FROM tbl1")
+    assertEquals(ResultKind.SUCCESS, tableResult2.getResultKind)
+    assertTrue(util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get()
+      
.tableExists(ObjectPath.fromString(s"${util.tableEnv.getCurrentDatabase}.view1")))
+
+    val tableResult3 = util.tableEnv.executeSql("DROP VIEW view1")
+    assertEquals(ResultKind.SUCCESS, tableResult3.getResultKind)
+    assertFalse(util.tableEnv.getCatalog(util.tableEnv.getCurrentCatalog).get()
+      
.tableExists(ObjectPath.fromString(s"${util.tableEnv.getCurrentDatabase}.view1")))
+  }
+
+  @Test
+  def testExecuteSqlWithShowViews(): Unit = {
+    val util = batchTestUtil()
+    val createTableStmt =
+      """
+        |CREATE TABLE tbl1 (
+        |  a bigint,
+        |  b int,
+        |  c varchar
+        |) with (
+        |  'connector' = 'COLLECTION',
+        |  'is-bounded' = 'false'
+        |)
+      """.stripMargin
+    val tableResult1 = util.tableEnv.executeSql(createTableStmt)
+    assertEquals(ResultKind.SUCCESS, tableResult1.getResultKind)
+
+    val tableResult2 = util.tableEnv.executeSql("CREATE VIEW view1 AS SELECT * 
FROM tbl1")
+    assertEquals(ResultKind.SUCCESS, tableResult2.getResultKind)
+
+    val tableResult3 = util.tableEnv.executeSql("SHOW VIEWS")
+    assertEquals(ResultKind.SUCCESS_WITH_CONTENT, tableResult3.getResultKind)
+    checkData(
+      util.tableEnv.listViews().map(Row.of(_)).toList.asJava.iterator(),
+      tableResult3.collect())

Review comment:
       add a test "CREATE TEMPORARY VIEW ..."

##########
File path: 
flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java
##########
@@ -756,6 +756,13 @@ default Table fromValues(DataType rowType, Object... 
values) {
         */
        String[] listTables();
 
+       /**
+        * Gets the names of all views available in the current namespace (the 
current database of the current catalog).
+        * It returns both temporary and permanent views.
+        * @return A list of the names of all registered views in the current 
database of the current catalog.
+        */

Review comment:
       add `@see #listTemporaryViews()`

##########
File path: flink-python/pyflink/table/tests/test_environment_completeness.py
##########
@@ -50,7 +50,8 @@ def excluded_methods(cls):
             'createFunction',
             'dropFunction',
             'createTemporaryFunction',
-            'dropTemporaryFunction'}
+            'dropTemporaryFunction',
+            'listViews'}

Review comment:
       remove this, and we should support `list_views` in table_environment.py




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