You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by bl...@apache.org on 2019/10/30 16:55:59 UTC
[flink] branch master updated: [FLINK-14534][table]
FunctionCatalog.getUserDefinedFunctions() should include temp functions
This is an automated email from the ASF dual-hosted git repository.
bli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 470fd2f [FLINK-14534][table] FunctionCatalog.getUserDefinedFunctions() should include temp functions
470fd2f is described below
commit 470fd2f66fe1da0b640ac5154fa6562740aa25f8
Author: bowen.li <bo...@gmail.com>
AuthorDate: Fri Oct 25 14:53:31 2019 -0700
[FLINK-14534][table] FunctionCatalog.getUserDefinedFunctions() should include temp functions
FunctionCatalog.getUserDefinedFunctions() should include temp functions.
This closes #9998.
---
.../client/gateway/local/ExecutionContextTest.java | 4 +--
.../client/gateway/local/LocalExecutorITCase.java | 2 +-
.../flink/table/catalog/FunctionCatalog.java | 36 ++++++++++++++++------
3 files changed, 29 insertions(+), 13 deletions(-)
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
index a3a9ce9..e8f1145 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
@@ -158,7 +158,7 @@ public class ExecutionContextTest {
public void testFunctions() throws Exception {
final ExecutionContext<?> context = createDefaultExecutionContext();
final TableEnvironment tableEnv = context.createEnvironmentInstance().getTableEnvironment();
- final String[] expected = new String[]{"scalarUDF", "tableUDF", "aggregateUDF"};
+ final String[] expected = new String[]{"scalarudf", "tableudf", "aggregateudf"};
final String[] actual = tableEnv.listUserDefinedFunctions();
Arrays.sort(expected);
Arrays.sort(actual);
@@ -225,7 +225,7 @@ public class ExecutionContextTest {
tableEnv.listTables());
assertArrayEquals(
- new String[]{"SourceTemporalTable", "ViewTemporalTable"},
+ new String[]{"sourcetemporaltable", "viewtemporaltable"},
tableEnv.listUserDefinedFunctions());
assertArrayEquals(
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java
index f4c80aa..923d141 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java
@@ -223,7 +223,7 @@ public class LocalExecutorITCase extends TestLogger {
final List<String> actualTables = executor.listUserDefinedFunctions(session);
- final List<String> expectedTables = Arrays.asList("aggregateUDF", "tableUDF", "scalarUDF");
+ final List<String> expectedTables = Arrays.asList("aggregateudf", "tableudf", "scalarudf");
assertEquals(expectedTables, actualTables);
}
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
index 6527e9f..2b3d6dc 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
@@ -193,14 +193,22 @@ public class FunctionCatalog implements FunctionLookup {
);
}
+ /**
+ * Get names of all user defined functions, including temp system functions, temp catalog functions and catalog functions
+ * in the current catalog and current database.
+ */
public String[] getUserDefinedFunctions() {
return getUserDefinedFunctionNames().toArray(new String[0]);
}
+ /**
+ * Get names of all functions, including temp system functions, system functions, temp catalog functions and catalog functions
+ * in the current catalog and current database.
+ */
public String[] getFunctions() {
Set<String> result = getUserDefinedFunctionNames();
- // Get built-in functions
+ // add system functions
result.addAll(moduleManager.listFunctions());
return result.toArray(new String[0]);
@@ -209,20 +217,28 @@ public class FunctionCatalog implements FunctionLookup {
private Set<String> getUserDefinedFunctionNames() {
Set<String> result = new HashSet<>();
- // Get functions in catalog
- Catalog catalog = catalogManager.getCatalog(catalogManager.getCurrentCatalog()).get();
+ // add temp system functions
+ result.addAll(tempSystemFunctions.keySet());
+
+ String currentCatalog = catalogManager.getCurrentCatalog();
+ String currentDatabase = catalogManager.getCurrentDatabase();
+
+ // add temp catalog functions
+ result.addAll(tempCatalogFunctions.keySet().stream()
+ .filter(oi -> oi.getCatalogName().equals(currentCatalog)
+ && oi.getDatabaseName().equals(currentDatabase))
+ .map(oi -> oi.getObjectName())
+ .collect(Collectors.toSet())
+ );
+
+ // add catalog functions
+ Catalog catalog = catalogManager.getCatalog(currentCatalog).get();
try {
- result.addAll(catalog.listFunctions(catalogManager.getCurrentDatabase()));
+ result.addAll(catalog.listFunctions(currentDatabase));
} catch (DatabaseNotExistException e) {
// Ignore since there will always be a current database of the current catalog
}
- // Get functions registered in memory
- result.addAll(
- tempSystemFunctions.values().stream()
- .map(FunctionDefinition::toString)
- .collect(Collectors.toSet()));
-
return result;
}