diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java index 51a56fb4edd63..63263002cbdcb 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java @@ -96,7 +96,7 @@ public void testInsertIntoNonPartitionTable() throws Exception { tableEnv.registerTable("src", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from src").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from src").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); verifyWrittenData(toWrite, hiveShell.executeQuery("select * from " + tblName)); @@ -138,7 +138,7 @@ public void testWriteComplexType() throws Exception { tableEnv.registerTable("complexSrc", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from complexSrc").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from complexSrc").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); List result = hiveShell.executeQuery("select * from " + tblName); @@ -177,7 +177,7 @@ public void testWriteNestedComplexType() throws Exception { Table src = tableEnv.fromTableSource(new CollectionTableSource(toWrite, rowTypeInfo)); tableEnv.registerTable("nestedSrc", src); tableEnv.registerCatalog("hive", hiveCatalog); - tableEnv.sqlQuery("select * from nestedSrc").insertInto("hive", "default", "dest"); + tableEnv.sqlQuery("select * from nestedSrc").insertInto("hive.`default`.dest"); tableEnv.execute("mytest"); List result = hiveShell.executeQuery("select * from " + tblName); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java index e274633472a51..2e06d519288ce 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java @@ -133,9 +133,7 @@ public static List collectTable(TableEnvironment tableEnv, Table table) thr sink.init(serializer, id); String sinkName = UUID.randomUUID().toString(); tableEnv.registerTableSink(sinkName, sink); - final String builtInCatalogName = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; - final String builtInDBName = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; - tableEnv.insertInto(table, builtInCatalogName, builtInDBName, sinkName); + tableEnv.insertInto(table, sinkName); JobExecutionResult result = tableEnv.execute("collect-table"); ArrayList data = result.getAccumulatorResult(id); return SerializedListAccumulator.deserializeList(data, serializer); diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java index d7231ab6142f6..dfa40a0d9b9e3 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java @@ -28,7 +28,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; @@ -564,8 +563,7 @@ private List collectBatchResult(Table table) throws Exception { schema.getFieldNames(), types.toArray(new TypeInformation[0])); return JavaScalaConversionUtil.toJava( BatchTableEnvUtil.collect( - t.getTableEnvironment(), table, configuredSink, Option.apply("JOB"), - EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, EnvironmentSettings.DEFAULT_BUILTIN_DATABASE)); + t.getTableEnvironment(), table, configuredSink, Option.apply("JOB"))); } } diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index 4ea3ec70b5aa3..2feaf525c1e55 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -608,7 +608,7 @@ def drop_columns(self, fields): """ return Table(self._j_table.dropColumns(fields)) - def insert_into(self, table_path, *table_path_continued): + def insert_into(self, table_path): """ Writes the :class:`Table` to a :class:`TableSink` that was registered under the specified name. For the path resolution algorithm see @@ -619,15 +619,10 @@ def insert_into(self, table_path, *table_path_continued): >>> tab.insert_into("sink") - :param table_path: The first part of the path of the registered :class:`TableSink` to which - the :class:`Table` is written. This is to ensure at least the name of the - :class:`Table` is provided. - :param table_path_continued: The remaining part of the path of the registered - :class:`TableSink` to which the :class:`Table` is written. + :param table_path: The path of the registered :class:`TableSink` to which + the :class:`Table` is written. """ - gateway = get_gateway() - j_table_path = to_jarray(gateway.jvm.String, table_path_continued) - self._j_table.insertInto(table_path, j_table_path) + self._j_table.insertInto(table_path) def get_schema(self): """ diff --git a/flink-python/pyflink/table/tests/test_environment_completeness.py b/flink-python/pyflink/table/tests/test_environment_completeness.py index d50bebbb4a3e4..4ff3d29273c13 100644 --- a/flink-python/pyflink/table/tests/test_environment_completeness.py +++ b/flink-python/pyflink/table/tests/test_environment_completeness.py @@ -48,7 +48,13 @@ def excluded_methods(cls): 'getCompletionHints', 'create', 'loadModule', - 'unloadModule'} + 'unloadModule', + 'listTemporaryTables', + 'createTemporaryView', + 'dropTemporaryTable', + 'listTemporaryViews', + 'from', + 'dropTemporaryView'} if __name__ == '__main__': diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index aff7e7daff00b..605a3661849e8 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -32,7 +32,6 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.QueryConfig; import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; @@ -494,8 +493,6 @@ private ResultDescriptor executeQueryInternal(ExecutionContext context, S envInst.getTableEnvironment().registerTableSink(jobName, result.getTableSink()); table.insertInto( envInst.getQueryConfig(), - EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, - EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, jobName); return null; }); 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 24799e047838c..f4c80aa6c2aad 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 @@ -145,13 +145,13 @@ public void testValidateSession() throws Exception { List actualTables = executor.listTables(session); List expectedTables = Arrays.asList( + "AdditionalView1", + "AdditionalView2", "TableNumber1", "TableNumber2", "TableSourceSink", "TestView1", - "TestView2", - "AdditionalView1", - "AdditionalView2"); + "TestView2"); assertEquals(expectedTables, actualTables); session.removeView("AdditionalView1"); @@ -183,8 +183,8 @@ public void testListCatalogs() throws Exception { final List actualCatalogs = executor.listCatalogs(session); final List expectedCatalogs = Arrays.asList( - "default_catalog", "catalog1", + "default_catalog", "simple-catalog"); assertEquals(expectedCatalogs, actualCatalogs); } diff --git a/flink-table/flink-table-api-java-bridge/pom.xml b/flink-table/flink-table-api-java-bridge/pom.xml index ea43eb15e1d77..79e24d0572065 100644 --- a/flink-table/flink-table-api-java-bridge/pom.xml +++ b/flink-table/flink-table-api-java-bridge/pom.xml @@ -52,5 +52,13 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} + + + org.apache.flink + flink-table-api-java + ${project.version} + test-jar + test + diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java index 38d0063c47887..42214c85ed0b7 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/BatchTableEnvironment.java @@ -109,24 +109,50 @@ public interface BatchTableEnvironment extends TableEnvironment { Table fromDataSet(DataSet dataSet, String fields); /** - * Registers the given {@link DataSet} as table in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataSet}. + * Registered views can be referenced in SQL queries. * - * The field names of the {@link Table} are automatically derived from the type of the{@link DataSet}. + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataSet}. + * + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataSet)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. * * @param name The name under which the {@link DataSet} is registered in the catalog. * @param dataSet The {@link DataSet} to register. * @param The type of the {@link DataSet} to register. + * @deprecated use {@link #createTemporaryView(String, DataSet)} */ + @Deprecated void registerDataSet(String name, DataSet dataSet); /** - * Registers the given {@link DataSet} as table with specified field names in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataSet} in a given path. + * Registered views can be referenced in SQL queries. * - * Example: + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataSet}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataSet The {@link DataSet} out of which to create the view. + * @param The type of the {@link DataSet}. + */ + void createTemporaryView(String path, DataSet dataSet); + + /** + * Creates a view from the given {@link DataSet} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: * *

 	 * {@code
@@ -135,13 +161,47 @@ public interface BatchTableEnvironment extends TableEnvironment {
 	 * }
 	 * 
* + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataSet)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataSet} is registered in the catalog. * @param dataSet The {@link DataSet} to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @param The type of the {@link DataSet} to register. + * @deprecated use {@link #createTemporaryView(String, DataSet, String)} */ + @Deprecated void registerDataSet(String name, DataSet dataSet, String fields); + /** + * Creates a view from the given {@link DataSet} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: + * + *

+	 * {@code
+	 *   DataSet> set = ...
+	 *   tableEnv.createTemporaryView("cat.db.myTable", set, "a, b");
+	 * }
+	 * 
+ * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataSet The {@link DataSet} out of which to create the view. + * @param fields The field names of the registered view. + * @param The type of the {@link DataSet}. + */ + void createTemporaryView(String path, DataSet dataSet, String fields); + /** * Converts the given {@link Table} into a {@link DataSet} of a specified type. * @@ -249,7 +309,9 @@ public interface BatchTableEnvironment extends TableEnvironment { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, BatchQueryConfig queryConfig, String sinkPath, String... sinkPathContinued); /** diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java index a18f3d4226d15..fd0051780f75e 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/StreamTableEnvironment.java @@ -39,7 +39,7 @@ import org.apache.flink.table.sinks.TableSink; /** - * This table environment is the entry point and central context for creating Table & SQL + * This table environment is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -60,7 +60,7 @@ public interface StreamTableEnvironment extends TableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -86,7 +86,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron } /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -117,7 +117,7 @@ static StreamTableEnvironment create( } /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs that integrate with the Java-specific {@link DataStream} API. * *

It is unified for bounded and unbounded data processing. @@ -164,7 +164,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * @param name The name under which the function is registered. * @param aggregateFunction The AggregateFunction to register. * @param The type of the output value. - * @tparam ACC The type of aggregate accumulator. + * @param The type of aggregate accumulator. */ void registerFunction(String name, AggregateFunction aggregateFunction); @@ -175,7 +175,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * @param name The name under which the function is registered. * @param tableAggregateFunction The TableAggregateFunction to register. * @param The type of the output value. - * @tparam ACC The type of aggregate accumulator. + * @param The type of aggregate accumulator. */ void registerFunction(String name, TableAggregateFunction tableAggregateFunction); @@ -194,7 +194,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron /** * Converts the given {@link DataStream} into a {@link Table} with specified field names. * - * Example: + *

Example: * *

 	 * {@code
@@ -211,43 +211,103 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron
 	 Table fromDataStream(DataStream dataStream, String fields);
 
 	/**
-	 * Registers the given {@link DataStream} as table in the {@link TableEnvironment}'s catalog.
-	 * Registered tables can be referenced in SQL queries.
+	 * Creates a view from the given {@link DataStream}.
+	 * Registered views can be referenced in SQL queries.
 	 *
-	 * The field names of the {@link Table} are automatically derived
+	 * 

The field names of the {@link Table} are automatically derived * from the type of the {@link DataStream}. * + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataStream)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataStream} is registered in the catalog. * @param dataStream The {@link DataStream} to register. * @param The type of the {@link DataStream} to register. + * @deprecated use {@link #createTemporaryView(String, DataStream)} */ + @Deprecated void registerDataStream(String name, DataStream dataStream); /** - * Registers the given {@link DataStream} as table with specified field names in the - * {@link TableEnvironment}'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given {@link DataStream} in a given path. + * Registered views can be referenced in SQL queries. + * + *

The field names of the {@link Table} are automatically derived + * from the type of the {@link DataStream}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the {@link DataStream} is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataStream The {@link DataStream} out of which to create the view. + * @param The type of the {@link DataStream}. + */ + void createTemporaryView(String path, DataStream dataStream); + + /** + * Creates a view from the given {@link DataStream} in a given path with specified field names. + * Registered views can be referenced in SQL queries. * - * Example: + *

Example: * *

 	 * {@code
-	 *   DataStream> set = ...
-	 *   tableEnv.registerDataStream("myTable", set, "a, b")
+	 *   DataStream> stream = ...
+	 *   tableEnv.registerDataStream("myTable", stream, "a, b")
 	 * }
 	 * 
* + *

The view is registered in the namespace of the current catalog and database. To register the view in + * a different catalog use {@link #createTemporaryView(String, DataStream)}. + * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link DataStream} is registered in the catalog. * @param dataStream The {@link DataStream} to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @param The type of the {@link DataStream} to register. + * @deprecated use {@link #createTemporaryView(String, DataStream, String)} */ + @Deprecated void registerDataStream(String name, DataStream dataStream, String fields); + /** + * Creates a view from the given {@link DataStream} in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + *

Example: + * + *

+	 * {@code
+	 *   DataStream> stream = ...
+	 *   tableEnv.createTemporaryView("cat.db.myTable", stream, "a, b")
+	 * }
+	 * 
+ * + *

Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the {@link DataStream} is created. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param dataStream The {@link DataStream} out of which to create the view. + * @param fields The field names of the created view. + * @param The type of the {@link DataStream}. + */ + void createTemporaryView(String path, DataStream dataStream, String fields); + /** * Converts the given {@link Table} into an append {@link DataStream} of a specified type. * - * The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified + *

The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified * by update or delete changes, the conversion will fail. * *

The fields of the {@link Table} are mapped to {@link DataStream} fields as follows: @@ -267,7 +327,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron /** * Converts the given {@link Table} into an append {@link DataStream} of a specified type. * - * The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified + *

The {@link Table} must only have insert (append) changes. If the {@link Table} is also modified * by update or delete changes, the conversion will fail. * *

The fields of the {@link Table} are mapped to {@link DataStream} fields as follows: @@ -443,7 +503,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * .field("count", "DECIMAL") * .field("proc-time", "TIMESTAMP").proctime()) * .inAppendMode() - * .registerSource("MyTable") + * .createTemporaryTable("MyTable") * } *

* @@ -489,7 +549,9 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, StreamQueryConfig queryConfig, String sinkPath, String... sinkPathContinued); /** diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java index 4addf4fe5e188..962b0d3cac7c8 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImpl.java @@ -209,12 +209,22 @@ public Table fromDataStream(DataStream dataStream, String fields) { @Override public void registerDataStream(String name, DataStream dataStream) { - registerTable(name, fromDataStream(dataStream)); + createTemporaryView(name, dataStream); + } + + @Override + public void createTemporaryView(String path, DataStream dataStream) { + createTemporaryView(path, fromDataStream(dataStream)); } @Override public void registerDataStream(String name, DataStream dataStream, String fields) { - registerTable(name, fromDataStream(dataStream, fields)); + createTemporaryView(name, dataStream, fields); + } + + @Override + public void createTemporaryView(String path, DataStream dataStream, String fields) { + createTemporaryView(path, fromDataStream(dataStream, fields)); } @Override diff --git a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java index 6b1724de54cac..1a0cf5a2b3089 100644 --- a/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java +++ b/flink-table/flink-table-api-java-bridge/src/test/java/org/apache/flink/table/api/java/internal/StreamTableEnvironmentImplTest.java @@ -18,7 +18,6 @@ package org.apache.flink.table.api.java.internal; -import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -28,11 +27,10 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.GenericInMemoryCatalog; -import org.apache.flink.table.delegation.Executor; -import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.module.ModuleManager; import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.utils.ExecutorMock; +import org.apache.flink.table.utils.PlannerMock; import org.apache.flink.types.Row; import org.junit.Test; @@ -101,48 +99,21 @@ private StreamTableEnvironmentImpl getStreamTableEnvironment( new TableConfig(), env, new TestPlanner(elements.getTransformation()), - executor, + new ExecutorMock(), true ); } - private static class TestPlanner implements Planner { + private static class TestPlanner extends PlannerMock { private final Transformation transformation; private TestPlanner(Transformation transformation) { this.transformation = transformation; } - @Override - public List parse(String statement) { - throw new AssertionError("Should not be called"); - } - @Override public List> translate(List modifyOperations) { return Collections.singletonList(transformation); } - - @Override - public String explain(List operations, boolean extended) { - throw new AssertionError("Should not be called"); - } - - @Override - public String[] getCompletionHints(String statement, int position) { - throw new AssertionError("Should not be called"); - } } - - private final Executor executor = new Executor() { - @Override - public void apply(List> transformations) { - - } - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - throw new AssertionError("Should not be called"); - } - }; } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java index 70350faa83ec7..c696c65ca31d5 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java @@ -821,12 +821,10 @@ public interface Table { * {@code org.apache.flink.table.sinks.RetractStreamTableSink}, or an * {@code org.apache.flink.table.sinks.UpsertStreamTableSink}. * - * @param tablePath The first part of the path of the registered {@link TableSink} to which the {@link Table} is - * written. This is to ensure at least the name of the {@link TableSink} is provided. - * @param tablePathContinued The remaining part of the path of the registered {@link TableSink} to which the - * {@link Table} is written. + * @param tablePath The path of the registered {@link TableSink} to which the {@link Table} is + * written. */ - void insertInto(String tablePath, String... tablePathContinued); + void insertInto(String tablePath); /** * Writes the {@link Table} to a {@link TableSink} that was registered under the specified name @@ -840,7 +838,7 @@ public interface Table { * * @param tableName The name of the {@link TableSink} to which the {@link Table} is written. * @param conf The {@link QueryConfig} to use. - * @deprecated use {@link #insertInto(QueryConfig, String, String...)} + * @deprecated use {@link #insertInto(String)} */ @Deprecated void insertInto(String tableName, QueryConfig conf); @@ -860,7 +858,9 @@ public interface Table { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param tablePathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String)} */ + @Deprecated void insertInto(QueryConfig conf, String tablePath, String... tablePathContinued); /** diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java index 344e7ffe796d8..a0a59fab58307 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java @@ -36,7 +36,7 @@ import java.util.Optional; /** - * A table environment is the base class, entry point, and central context for creating Table & SQL + * A table environment is the base class, entry point, and central context for creating Table and SQL * API programs. * *

It is unified both on a language level for all JVM-based languages (i.e. there is no distinction @@ -50,6 +50,14 @@ *

  • Offering further configuration options.
  • * * + *

    The path in methods such as {@link #createTemporaryView(String, Table)} should be a proper SQL identifier. + * The syntax is following [[catalog-name.]database-name.]object-name, where the catalog name and database are + * optional. For path resolution see {@link #useCatalog(String)} and {@link #useDatabase(String)}. All keywords + * or other special characters need to be escaped. + * + *

    Example: `cat.1`.`db`.`Table` resolves to an object named 'Table' (table is a reserved keyword, thus must + * be escaped) in a catalog named 'cat.1' and database named 'db'. + * *

    Note: This environment is meant for pure table programs. If you would like to convert from or to * other Flink APIs, it might be necessary to use one of the available language-specific table environments * in the corresponding bridging modules. @@ -58,7 +66,7 @@ public interface TableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & SQL + * Creates a table environment that is the entry point and central context for creating Table and SQL * API programs. * *

    It is unified both on a language level for all JVM-based languages (i.e. there is no distinction @@ -133,18 +141,43 @@ static TableEnvironment create(EnvironmentSettings settings) { * Registers a {@link Table} under a unique name in the TableEnvironment's catalog. * Registered tables can be referenced in SQL queries. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the table will be registered. * @param table The table to register. + * @deprecated use {@link #createTemporaryView(String, Table)} */ + @Deprecated void registerTable(String name, Table table); + /** + * Registers a {@link Table} API object as a temporary view similar to SQL temporary views. + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + * @param path The path under which the view will be registered. + * See also the {@link TableEnvironment} class description for the format of the path. + * @param view The view to register. + */ + void createTemporaryView(String path, Table view); + /** * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog. * Registered tables can be referenced in SQL queries. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSource} is registered. * @param tableSource The {@link TableSource} to register. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ + @Deprecated void registerTableSource(String name, TableSource tableSource); /** @@ -152,11 +185,15 @@ static TableEnvironment create(EnvironmentSettings settings) { * {@link TableEnvironment}'s catalog. * Registered sink tables can be referenced in SQL DML statements. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSink} is registered. * @param fieldNames The field names to register with the {@link TableSink}. * @param fieldTypes The field types to register with the {@link TableSink}. * @param tableSink The {@link TableSink} to register. - * @deprecated Use {@link #registerTableSink(String, TableSink)} instead. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ @Deprecated void registerTableSink(String name, String[] fieldNames, TypeInformation[] fieldTypes, TableSink tableSink); @@ -166,9 +203,15 @@ static TableEnvironment create(EnvironmentSettings settings) { * this {@link TableEnvironment}'s catalog. * Registered sink tables can be referenced in SQL DML statements. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name The name under which the {@link TableSink} is registered. * @param configuredSink The configured {@link TableSink} to register. + * @deprecated Use {@link #connect(ConnectorDescriptor)} instead. */ + @Deprecated void registerTableSink(String name, TableSink configuredSink); /** @@ -200,9 +243,50 @@ static TableEnvironment create(EnvironmentSettings settings) { * @return The resulting {@link Table}. * @see TableEnvironment#useCatalog(String) * @see TableEnvironment#useDatabase(String) + * @deprecated use {@link #from(String)} */ + @Deprecated Table scan(String... tablePath); + /** + * Reads a registered table and returns the resulting {@link Table}. + * + *

    A table to scan must be registered in the {@link TableEnvironment}. + * + *

    See the documentation of {@link TableEnvironment#useDatabase(String)} or + * {@link TableEnvironment#useCatalog(String)} for the rules on the path resolution. + * + *

    Examples: + * + *

    Reading a table from default catalog and database. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("tableName");
    +	 * }
    +	 * 
    + * + *

    Reading a table from a registered catalog. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("catalogName.dbName.tableName");
    +	 * }
    +	 * 
    + * + *

    Reading a table from a registered catalog with escaping. ({@code Table} is a reserved keyword). + * Dots in e.g. a database name also must be escaped. + *

    +	 * {@code
    +	 *   Table tab = tableEnv.from("catalogName.`db.Name`.`Table`");
    +	 * }
    +	 * 
    + * + * @param path The path of a table API object to scan. + * @return Either a table or virtual table (=view). + * @see TableEnvironment#useCatalog(String) + * @see TableEnvironment#useDatabase(String) + */ + Table from(String path); + /** * Writes the {@link Table} to a {@link TableSink} that was registered under the specified name. * @@ -214,9 +298,22 @@ static TableEnvironment create(EnvironmentSettings settings) { * written. This is to ensure at least the name of the {@link TableSink} is provided. * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the * {@link Table} is written. + * @deprecated use {@link #insertInto(String, Table)} */ + @Deprecated void insertInto(Table table, String sinkPath, String... sinkPathContinued); + /** + * Instructs to write the content of a {@link Table} API object into a table. + * + *

    See the documentation of {@link TableEnvironment#useDatabase(String)} or + * {@link TableEnvironment#useCatalog(String)} for the rules on the path resolution. + * + * @param targetPath The path of the registered {@link TableSink} to which the {@link Table} is written. + * @param table The Table to write to the sink. + */ + void insertInto(String targetPath, Table table); + /** * Creates a table source and/or table sink from a descriptor. * @@ -272,12 +369,35 @@ static TableEnvironment create(EnvironmentSettings settings) { String[] listDatabases(); /** - * Gets the names of all tables registered in the current database of the current catalog. + * Gets the names of all tables available in the current namespace (the current database of the current catalog). + * It returns both temporary and permanent tables and views. * * @return A list of the names of all registered tables in the current database of the current catalog. + * @see #listTemporaryTables() + * @see #listTemporaryViews() */ String[] listTables(); + /** + * Gets the names of all temporary tables and views available in the current namespace (the current + * database of the current catalog). + * + * @return A list of the names of all registered temporary tables and views in the current database + * of the current catalog. + * @see #listTables() + */ + String[] listTemporaryTables(); + + /** + * Gets the names of all temporary views available in the current namespace (the current + * database of the current catalog). + * + * @return A list of the names of all registered temporary views in the current database + * of the current catalog. + * @see #listTables() + */ + String[] listTemporaryViews(); + /** * Gets the names of all user defined functions registered in this environment. */ @@ -288,6 +408,26 @@ static TableEnvironment create(EnvironmentSettings settings) { */ String[] listFunctions(); + /** + * Drops a temporary table registered in the given path. + * + *

    If a permanent table with a given path exists, it will be used + * from now on for any queries that reference this path. + * + * @return true if a table existed in the given path and was removed + */ + boolean dropTemporaryTable(String path); + + /** + * Drops a temporary view registered in the given path. + * + *

    If a permanent table or view with a given path exists, it will be used + * from now on for any queries that reference this path. + * + * @return true if a view existed in the given path and was removed + */ + boolean dropTemporaryView(String path); + /** * Returns the AST of the specified Table API and SQL queries and the execution plan to compute * the result of the given {@link Table}. diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java new file mode 100644 index 0000000000000..f2bc4eca8458b --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/Registration.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api.internal; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; + +/** + * A way to register a table in a {@link TableEnvironment} that this descriptor originates from. + */ +@Internal +public interface Registration { + /** + * Registers an external {@link TableSource} in this {@link TableEnvironment}'s catalog. + * Registered tables can be referenced in SQL queries. + * + * @param name The name under which the {@link TableSource} is registered. + * @param tableSource The {@link TableSource} to register. + * @see TableEnvironment#registerTableSource(String, TableSource) + */ + void createTableSource(String name, TableSource tableSource); + + /** + * Registers an external {@link TableSink} with already configured field names and field types in + * this {@link TableEnvironment}'s catalog. + * Registered sink tables can be referenced in SQL DML statements. + * + * @param name The name under which the {@link TableSink} is registered. + * @param tableSink The configured {@link TableSink} to register. + * @see TableEnvironment#registerTableSink(String, TableSink) + */ + void createTableSink(String name, TableSink tableSink); + + /** + * Creates a temporary table in a given path. + * + * @param path Path where to register the given table + * @param table table to register + */ + void createTemporaryTable(String path, CatalogBaseTable table); +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index d13dea35562b3..10459d07346b9 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.SqlParserException; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; @@ -37,9 +38,10 @@ import org.apache.flink.table.catalog.GenericInMemoryCatalog; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.QueryOperationCatalogView; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.ExecutorFactory; +import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.delegation.PlannerFactory; import org.apache.flink.table.descriptors.ConnectTableDescriptor; @@ -92,6 +94,30 @@ public class TableEnvironmentImpl implements TableEnvironment { protected final Executor execEnv; protected final FunctionCatalog functionCatalog; protected final Planner planner; + protected final Parser parser; + + /** + * Provides necessary methods for {@link ConnectTableDescriptor}. + */ + private final Registration registration = new Registration() { + @Override + public void createTemporaryTable(String path, CatalogBaseTable table) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier( + unresolvedIdentifier); + catalogManager.createTemporaryTable(table, objectIdentifier, false); + } + + @Override + public void createTableSource(String name, TableSource tableSource) { + registerTableSource(name, tableSource); + } + + @Override + public void createTableSink(String name, TableSink tableSink) { + registerTableSink(name, tableSink); + } + }; protected TableEnvironmentImpl( CatalogManager catalogManager, @@ -109,11 +135,21 @@ protected TableEnvironmentImpl( this.functionCatalog = functionCatalog; this.planner = planner; + this.parser = planner.getParser(); this.operationTreeBuilder = OperationTreeBuilder.create( functionCatalog, path -> { - Optional catalogTableOperation = scanInternal(path); - return catalogTableOperation.map(tableOperation -> new TableReferenceExpression(path, tableOperation)); + try { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + Optional catalogQueryOperation = scanInternal(unresolvedIdentifier); + return catalogQueryOperation.map(t -> new TableReferenceExpression(path, t)); + } catch (SqlParserException ex) { + // The TableLookup is used during resolution of expressions and it actually might not be an + // identifier of a table. It might be a reference to some other object such as column, local + // reference etc. This method should return empty optional in such cases to fallback for other + // identifiers resolution. + return Optional.empty(); + } }, isStreamingMode ); @@ -189,13 +225,26 @@ public void registerFunction(String name, ScalarFunction function) { @Override public void registerTable(String name, Table table) { - if (((TableImpl) table).getTableEnvironment() != this) { + UnresolvedIdentifier identifier = UnresolvedIdentifier.of(name); + createTemporaryView(identifier, table); + } + + @Override + public void createTemporaryView(String path, Table view) { + UnresolvedIdentifier identifier = parser.parseIdentifier(path); + createTemporaryView(identifier, view); + } + + private void createTemporaryView(UnresolvedIdentifier identifier, Table view) { + if (((TableImpl) view).getTableEnvironment() != this) { throw new TableException( - "Only tables that belong to this TableEnvironment can be registered."); + "Only table API objects that belong to this TableEnvironment can be registered."); } - CatalogBaseTable tableTable = new QueryOperationCatalogView(table.getQueryOperation()); - catalogManager.createTable(tableTable, getTemporaryObjectIdentifier(name), false); + CatalogBaseTable tableTable = new QueryOperationCatalogView(view.getQueryOperation()); + + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(identifier); + catalogManager.createTemporaryTable(tableTable, tableIdentifier, false); } @Override @@ -226,26 +275,72 @@ public void registerTableSink(String name, TableSink configuredSink) { @Override public Table scan(String... tablePath) { - return scanInternal(tablePath).map(this::createTable) + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(tablePath); + return scanInternal(unresolvedIdentifier) + .map(this::createTable) .orElseThrow(() -> new ValidationException(String.format( - "Table '%s' was not found.", - String.join(".", tablePath)))); + "Table %s was not found.", + unresolvedIdentifier))); } - private Optional scanInternal(String... tablePath) { - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(tablePath); - return catalogManager.getTable(objectIdentifier) - .map(t -> new CatalogQueryOperation(objectIdentifier, t.getSchema())); + @Override + public Table from(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return scanInternal(unresolvedIdentifier) + .map(this::createTable) + .orElseThrow(() -> new ValidationException(String.format( + "Table %s was not found.", + unresolvedIdentifier))); + } + + @Override + public void insertInto(String targetPath, Table table) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(targetPath); + insertIntoInternal(unresolvedIdentifier, table); + } + + @Override + public void insertInto(Table table, String sinkPath, String... sinkPathContinued) { + List fullPath = new ArrayList<>(Arrays.asList(sinkPathContinued)); + fullPath.add(0, sinkPath); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(fullPath.toArray(new String[0])); + + insertIntoInternal(unresolvedIdentifier, table); + } + + private void insertIntoInternal(UnresolvedIdentifier unresolvedIdentifier, Table table) { + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + List modifyOperations = Collections.singletonList( + new CatalogSinkModifyOperation( + objectIdentifier, + table.getQueryOperation())); + + if (isEagerOperationTranslation()) { + translate(modifyOperations); + } else { + buffer(modifyOperations); + } + } + + private Optional scanInternal(UnresolvedIdentifier identifier) { + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(identifier); + + return catalogManager.getTable(tableIdentifier) + .map(t -> new CatalogQueryOperation(tableIdentifier, t.getTable().getSchema())); } @Override public ConnectTableDescriptor connect(ConnectorDescriptor connectorDescriptor) { - return new StreamTableDescriptor(this, connectorDescriptor); + return new StreamTableDescriptor(registration, connectorDescriptor); } @Override public String[] listCatalogs() { - return catalogManager.getCatalogs().toArray(new String[0]); + return catalogManager.listCatalogs() + .stream() + .sorted() + .toArray(String[]::new); } @Override @@ -263,17 +358,38 @@ public String[] listDatabases() { @Override public String[] listTables() { - String currentCatalogName = catalogManager.getCurrentCatalog(); - Optional currentCatalog = catalogManager.getCatalog(currentCatalogName); - - return currentCatalog.map(catalog -> { - try { - return catalog.listTables(catalogManager.getCurrentDatabase()).toArray(new String[0]); - } catch (DatabaseNotExistException e) { - throw new ValidationException("Current database does not exist", e); - } - }).orElseThrow(() -> - new TableException(String.format("The current catalog %s does not exist.", currentCatalogName))); + return catalogManager.listTables() + .stream() + .sorted() + .toArray(String[]::new); + } + + @Override + public String[] listTemporaryTables() { + return catalogManager.listTemporaryTables() + .stream() + .sorted() + .toArray(String[]::new); + } + + @Override + public String[] listTemporaryViews() { + return catalogManager.listTemporaryViews() + .stream() + .sorted() + .toArray(String[]::new); + } + + @Override + public boolean dropTemporaryTable(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return catalogManager.dropTemporaryTable(unresolvedIdentifier); + } + + @Override + public boolean dropTemporaryView(String path) { + UnresolvedIdentifier unresolvedIdentifier = parser.parseIdentifier(path); + return catalogManager.dropTemporaryView(unresolvedIdentifier); } @Override @@ -310,7 +426,7 @@ public String[] getCompletionHints(String statement, int position) { @Override public Table sqlQuery(String query) { - List operations = planner.parse(query); + List operations = parser.parse(query); if (operations.size() != 1) { throw new ValidationException( @@ -328,26 +444,9 @@ public Table sqlQuery(String query) { } } - @Override - public void insertInto(Table table, String path, String... pathContinued) { - List fullPath = new ArrayList<>(Arrays.asList(pathContinued)); - fullPath.add(0, path); - - List modifyOperations = Collections.singletonList( - new CatalogSinkModifyOperation( - fullPath, - table.getQueryOperation())); - - if (isEagerOperationTranslation()) { - translate(modifyOperations); - } else { - buffer(modifyOperations); - } - } - @Override public void sqlUpdate(String stmt) { - List operations = planner.parse(stmt); + List operations = parser.parse(stmt); if (operations.size() != 1) { throw new TableException( @@ -366,15 +465,15 @@ public void sqlUpdate(String stmt) { } } else if (operation instanceof CreateTableOperation) { CreateTableOperation createTableOperation = (CreateTableOperation) operation; - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(createTableOperation.getTablePath()); catalogManager.createTable( createTableOperation.getCatalogTable(), - objectIdentifier, + createTableOperation.getTableIdentifier(), createTableOperation.isIgnoreIfExists()); } else if (operation instanceof DropTableOperation) { DropTableOperation dropTableOperation = (DropTableOperation) operation; - ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(dropTableOperation.getTableName()); - catalogManager.dropTable(objectIdentifier, dropTableOperation.isIfExists()); + catalogManager.dropTable( + dropTableOperation.getTableIdentifier(), + dropTableOperation.isIfExists()); } else { throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " + @@ -449,19 +548,10 @@ private void buffer(List modifyOperations) { bufferedModifyOperations.addAll(modifyOperations); } - private ObjectIdentifier getTemporaryObjectIdentifier(String name) { - return catalogManager.qualifyIdentifier( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); - } - private void registerTableSourceInternal(String name, TableSource tableSource) { validateTableSource(tableSource); - Optional table = getCatalogTable( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name)); + Optional table = getTemporaryTable(objectIdentifier); if (table.isPresent()) { if (table.get() instanceof ConnectorCatalogTable) { @@ -475,7 +565,7 @@ private void registerTableSourceInternal(String name, TableSource tableSource tableSource, sourceSinkTable.getTableSink().get(), !IS_STREAM_TABLE); - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sourceAndSink, objectIdentifier, true); } } else { throw new ValidationException(String.format( @@ -483,15 +573,13 @@ private void registerTableSourceInternal(String name, TableSource tableSource } } else { ConnectorCatalogTable source = ConnectorCatalogTable.source(tableSource, !IS_STREAM_TABLE); - catalogManager.createTable(source, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(source, objectIdentifier, false); } } private void registerTableSinkInternal(String name, TableSink tableSink) { - Optional table = getCatalogTable( - catalogManager.getBuiltInCatalogName(), - catalogManager.getBuiltInDatabaseName(), - name); + ObjectIdentifier objectIdentifier = catalogManager.qualifyIdentifier(UnresolvedIdentifier.of(name)); + Optional table = getTemporaryTable(objectIdentifier); if (table.isPresent()) { if (table.get() instanceof ConnectorCatalogTable) { @@ -503,7 +591,7 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { // wrapper contains only sink (not source) ConnectorCatalogTable sourceAndSink = ConnectorCatalogTable .sourceAndSink(sourceSinkTable.getTableSource().get(), tableSink, !IS_STREAM_TABLE); - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sourceAndSink, objectIdentifier, true); } } else { throw new ValidationException(String.format( @@ -511,12 +599,14 @@ private void registerTableSinkInternal(String name, TableSink tableSink) { } } else { ConnectorCatalogTable sink = ConnectorCatalogTable.sink(tableSink, !IS_STREAM_TABLE); - catalogManager.createTable(sink, getTemporaryObjectIdentifier(name), false); + catalogManager.createTemporaryTable(sink, objectIdentifier, false); } } - private Optional getCatalogTable(String... name) { - return catalogManager.getTable(catalogManager.qualifyIdentifier(name)); + private Optional getTemporaryTable(ObjectIdentifier identifier) { + return catalogManager.getTable(identifier) + .filter(CatalogManager.TableLookupResult::isTemporary) + .map(CatalogManager.TableLookupResult::getTable); } protected TableImpl createTable(QueryOperation tableOperation) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java index 41e483bb38c51..2b545abd739dc 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableImpl.java @@ -407,8 +407,8 @@ public Table fetch(int fetch) { } @Override - public void insertInto(String tablePath, String... tablePathContinued) { - tableEnvironment.insertInto(this, tablePath, tablePathContinued); + public void insertInto(String tablePath) { + tableEnvironment.insertInto(tablePath, this); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java index 564770978f69b..f50c00ff27ad7 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/CatalogManager.java @@ -31,11 +31,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static java.lang.String.format; import static org.apache.flink.util.Preconditions.checkArgument; @@ -52,6 +56,10 @@ public class CatalogManager { // A map between names and catalogs. private Map catalogs; + // Those tables take precedence over corresponding permanent tables, thus they shadow + // tables coming from catalogs. + private Map temporaryTables; + // The name of the current catalog and database private String currentCatalogName; @@ -70,6 +78,7 @@ public CatalogManager(String defaultCatalogName, Catalog defaultCatalog) { this.currentCatalogName = defaultCatalogName; this.currentDatabaseName = defaultCatalog.getDefaultDatabase(); + this.temporaryTables = new HashMap<>(); // right now the default catalog is always the built-in one this.builtInCatalogName = defaultCatalogName; } @@ -103,20 +112,11 @@ public Optional getCatalog(String catalogName) { return Optional.ofNullable(catalogs.get(catalogName)); } - /** - * Retrieves names of all registered catalogs. - * - * @return a set of names of registered catalogs - */ - public Set getCatalogs() { - return catalogs.keySet(); - } - /** * Gets the current catalog that will be used when resolving table path. * * @return the current catalog - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public String getCurrentCatalog() { return currentCatalogName; @@ -127,7 +127,7 @@ public String getCurrentCatalog() { * * @param catalogName catalog name to set as current catalog * @throws CatalogNotExistException thrown if the catalog doesn't exist - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public void setCurrentCatalog(String catalogName) throws CatalogNotExistException { checkArgument(!StringUtils.isNullOrWhitespaceOnly(catalogName), "Catalog name cannot be null or empty."); @@ -152,7 +152,7 @@ public void setCurrentCatalog(String catalogName) throws CatalogNotExistExceptio * Gets the current database name that will be used when resolving table path. * * @return the current database - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) */ public String getCurrentDatabase() { return currentDatabaseName; @@ -164,7 +164,7 @@ public String getCurrentDatabase() { * * @param databaseName database name to set as current database name * @throws CatalogException thrown if the database doesn't exist in the current catalog - * @see CatalogManager#qualifyIdentifier(String...) + * @see CatalogManager#qualifyIdentifier(UnresolvedIdentifier) * @see CatalogManager#setCurrentCatalog(String) */ public void setCurrentDatabase(String databaseName) { @@ -208,66 +208,236 @@ public String getBuiltInDatabaseName() { return catalogs.get(getBuiltInCatalogName()).getDefaultDatabase(); } + /** + * Result of a lookup for a table through {@link #getTable(ObjectIdentifier)}. It combines the + * {@link CatalogBaseTable} with additional information such as if the table is a temporary table or comes + * from the catalog. + */ + public static class TableLookupResult { + private final boolean isTemporary; + private final CatalogBaseTable table; + + private static TableLookupResult temporary(CatalogBaseTable table) { + return new TableLookupResult(true, table); + } + + private static TableLookupResult permanent(CatalogBaseTable table) { + return new TableLookupResult(false, table); + } + + private TableLookupResult(boolean isTemporary, CatalogBaseTable table) { + this.isTemporary = isTemporary; + this.table = table; + } + + public boolean isTemporary() { + return isTemporary; + } + + public CatalogBaseTable getTable() { + return table; + } + } + /** * Retrieves a fully qualified table. If the path is not yet fully qualified use - * {@link #qualifyIdentifier(String...)} first. + * {@link #qualifyIdentifier(UnresolvedIdentifier)} first. * * @param objectIdentifier full path of the table to retrieve * @return table that the path points to. */ - public Optional getTable(ObjectIdentifier objectIdentifier) { + public Optional getTable(ObjectIdentifier objectIdentifier) { try { - Catalog currentCatalog = catalogs.get(objectIdentifier.getCatalogName()); - ObjectPath objectPath = new ObjectPath( - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - - if (currentCatalog != null && currentCatalog.tableExists(objectPath)) { - return Optional.of(currentCatalog.getTable(objectPath)); + CatalogBaseTable temporaryTable = temporaryTables.get(objectIdentifier); + if (temporaryTable != null) { + return Optional.of(TableLookupResult.temporary(temporaryTable)); + } else { + return getPermanentTable(objectIdentifier); } } catch (TableNotExistException ignored) { } return Optional.empty(); } + private Optional getPermanentTable(ObjectIdentifier objectIdentifier) + throws TableNotExistException { + Catalog currentCatalog = catalogs.get(objectIdentifier.getCatalogName()); + ObjectPath objectPath = objectIdentifier.toObjectPath(); + + if (currentCatalog != null && currentCatalog.tableExists(objectPath)) { + return Optional.of(TableLookupResult.permanent(currentCatalog.getTable(objectPath))); + } + return Optional.empty(); + } + /** - * Returns the full name of the given table path, this name may be padded - * with current catalog/database name based on the {@code paths} length. + * Retrieves names of all registered catalogs. * - * @param path Table path whose format can be "catalog.db.table", "db.table" or "table" - * @return An array of complete table path + * @return a set of names of registered catalogs */ - public ObjectIdentifier qualifyIdentifier(String... path) { - if (path == null) { - throw new ValidationException("Table paths can not be null!"); - } - if (path.length < 1 || path.length > 3) { - throw new ValidationException("Table paths length must be " + - "between 1(inclusive) and 3(inclusive)"); - } - if (Arrays.stream(path).anyMatch(StringUtils::isNullOrWhitespaceOnly)) { - throw new ValidationException("Table paths contain null or " + - "while-space-only string"); - } + public Set listCatalogs() { + return Collections.unmodifiableSet(catalogs.keySet()); + } - String catalogName; - String dbName; - String tableName; - if (path.length == 3) { - catalogName = path[0]; - dbName = path[1]; - tableName = path[2]; - } else if (path.length == 2) { - catalogName = getCurrentCatalog(); - dbName = path[0]; - tableName = path[1]; - } else { - catalogName = getCurrentCatalog(); - dbName = getCurrentDatabase(); - tableName = path[0]; + /** + * Returns an array of names of all tables (tables and views, both temporary and permanent) + * registered in the namespace of the current catalog and database. + * + * @return names of all registered tables + */ + public Set listTables() { + return listTables(getCurrentCatalog(), getCurrentDatabase()); + } + + /** + * Returns an array of names of all tables (tables and views, both temporary and permanent) + * registered in the namespace of the current catalog and database. + * + * @return names of all registered tables + */ + public Set listTables(String catalogName, String databaseName) { + Catalog currentCatalog = catalogs.get(getCurrentCatalog()); + + try { + return Stream.concat( + currentCatalog.listTables(getCurrentDatabase()).stream(), + listTemporaryTablesInternal(catalogName, databaseName).map(e -> e.getKey().getObjectName()) + ).collect(Collectors.toSet()); + } catch (DatabaseNotExistException e) { + throw new ValidationException("Current database does not exist", e); } + } + + /** + * Returns an array of names of temporary tables registered in the namespace of the current + * catalog and database. + * + * @return names of registered temporary tables + */ + public Set listTemporaryTables() { + return listTemporaryTablesInternal(getCurrentCatalog(), getCurrentDatabase()) + .map(e -> e.getKey().getObjectName()) + .collect(Collectors.toSet()); + } + + /** + * Returns an array of names of temporary views registered in the namespace of the current + * catalog and database. + * + * @return names of registered temporary views + */ + public Set listTemporaryViews() { + return listTemporaryTablesInternal(getCurrentCatalog(), getCurrentDatabase()) + .filter(e -> e.getValue() instanceof CatalogView) + .map(e -> e.getKey().getObjectName()) + .collect(Collectors.toSet()); + } + + private Stream> listTemporaryTablesInternal( + String catalogName, + String databaseName) { + return temporaryTables + .entrySet() + .stream() + .filter(e -> { + ObjectIdentifier identifier = e.getKey(); + return identifier.getCatalogName().equals(catalogName) && + identifier.getDatabaseName().equals(databaseName); + }); + } + + /** + * Lists all available schemas in the root of the catalog manager. It is not equivalent to listing all catalogs + * as it includes also different catalog parts of the temporary objects. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @return list of schemas in the root of catalog manager + */ + public Set listSchemas() { + return Stream.concat( + catalogs.keySet().stream(), + temporaryTables.keySet().stream().map(ObjectIdentifier::getCatalogName) + ).collect(Collectors.toSet()); + } + + /** + * Lists all available schemas in the given catalog. It is not equivalent to listing databases within + * the given catalog as it includes also different database parts of the temporary objects identifiers. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @return list of schemas with the given prefix + */ + public Set listSchemas(String catalogName) { + return Stream.concat( + Optional.ofNullable(catalogs.get(catalogName)) + .map(Catalog::listDatabases) + .orElse(Collections.emptyList()) + .stream(), + temporaryTables.keySet() + .stream() + .filter(i -> i.getCatalogName().equals(catalogName)) + .map(ObjectIdentifier::getDatabaseName) + ).collect(Collectors.toSet()); + } + + /** + * Checks if there is a catalog with given name or is there a temporary object registered within a + * given catalog. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @return true if a subschema exists + */ + public boolean schemaExists(String catalogName) { + return getCatalog(catalogName).isPresent() || + temporaryTables.keySet() + .stream() + .anyMatch(i -> i.getCatalogName().equals(catalogName)); + } + + /** + * Checks if there is a database with given name in a given catalog or is there a temporary + * object registered within a given catalog and database. + * + * NOTE:It is primarily used for interacting with Calcite's schema. + * + * @param catalogName filter for the catalog part of the schema + * @param databaseName filter for the database part of the schema + * @return true if a subschema exists + */ + public boolean schemaExists(String catalogName, String databaseName) { + return temporaryDatabaseExists(catalogName, databaseName) || permanentDatabaseExists(catalogName, databaseName); + } + + private boolean temporaryDatabaseExists(String catalogName, String databaseName) { + return temporaryTables + .keySet() + .stream() + .anyMatch(i -> i.getCatalogName().equals(catalogName) && i.getDatabaseName().equals(databaseName)); + } + + private boolean permanentDatabaseExists(String catalogName, String databaseName) { + return getCatalog(catalogName) + .map(c -> c.databaseExists(databaseName)) + .orElse(false); + } - return ObjectIdentifier.of(catalogName, dbName, tableName); + /** + * Returns the full name of the given table path, this name may be padded + * with current catalog/database name based on the {@code identifier's} length. + * + * @param identifier an unresolved identifier + * @return a fully qualified object identifier + */ + public ObjectIdentifier qualifyIdentifier(UnresolvedIdentifier identifier) { + return ObjectIdentifier.of( + identifier.getCatalogName().orElseGet(this::getCurrentCatalog), + identifier.getDatabaseName().orElseGet(this::getCurrentDatabase), + identifier.getObjectName()); } /** @@ -285,13 +455,69 @@ public void createTable(CatalogBaseTable table, ObjectIdentifier objectIdentifie "CreateTable"); } + /** + * Creates a temporary table in a given fully qualified path. + * + * @param table The table to put in the given path. + * @param objectIdentifier The fully qualified path where to put the table. + * @param replace controls what happens if a table exists in the given path, + * if true the table is replaced, an exception will be thrown otherwise + */ + public void createTemporaryTable( + CatalogBaseTable table, + ObjectIdentifier objectIdentifier, + boolean replace) { + temporaryTables.compute(objectIdentifier, (k, v) -> { + if (v != null && !replace) { + throw new ValidationException(String.format("Temporary table %s already exists", objectIdentifier)); + } else { + return table; + } + }); + } + + /** + * Qualifies the given {@link UnresolvedIdentifier} with current catalog & database and + * removes a temporary table registered with this path if it exists. + * + * @param identifier potentially unresolved identifier + * @return true if a table with a given identifier existed and was removed, false otherwise + */ + public boolean dropTemporaryTable(UnresolvedIdentifier identifier) { + return dropTemporaryTableInternal(identifier, (table) -> table instanceof CatalogTable); + } + + /** + * Qualifies the given {@link UnresolvedIdentifier} with current catalog & database and + * removes a temporary view registered with this path if it exists. + * + * @param identifier potentially unresolved identifier + * @return true if a view with a given identifier existed and was removed, false otherwise + */ + public boolean dropTemporaryView(UnresolvedIdentifier identifier) { + return dropTemporaryTableInternal(identifier, (table) -> table instanceof CatalogView); + } + + private boolean dropTemporaryTableInternal( + UnresolvedIdentifier unresolvedIdentifier, + Predicate filter) { + ObjectIdentifier objectIdentifier = qualifyIdentifier(unresolvedIdentifier); + CatalogBaseTable catalogBaseTable = temporaryTables.get(objectIdentifier); + if (filter.test(catalogBaseTable)) { + temporaryTables.remove(objectIdentifier); + return true; + } else { + return false; + } + } + /** * Alters a table in a given fully qualified path. * * @param table The table to put in the given path * @param objectIdentifier The fully qualified path where to alter the table. * @param ignoreIfNotExists If false exception will be thrown if the table or database or catalog to be altered - * does not exist. + * does not exist. */ public void alterTable(CatalogBaseTable table, ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { execute( @@ -306,9 +532,14 @@ public void alterTable(CatalogBaseTable table, ObjectIdentifier objectIdentifier * * @param objectIdentifier The fully qualified path of the table to drop. * @param ignoreIfNotExists If false exception will be thrown if the table or database or catalog to be altered - * does not exist. + * does not exist. */ public void dropTable(ObjectIdentifier objectIdentifier, boolean ignoreIfNotExists) { + if (temporaryTables.containsKey(objectIdentifier)) { + throw new ValidationException(String.format( + "Temporary table with identifier '%s' exists. Drop it first before removing the permanent table.", + objectIdentifier)); + } execute( (catalog, path) -> catalog.dropTable(path, ignoreIfNotExists), objectIdentifier, diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java new file mode 100644 index 0000000000000..f6e9cc15391f9 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/UnresolvedIdentifier.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.utils.EncodingUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Identifier of an object, such as table, view, function or type in a catalog. This identifier + * cannot be used directly to access an object in a {@link CatalogManager}, but has to be first + * fully resolved into {@link ObjectIdentifier}. + */ +@Internal +public class UnresolvedIdentifier { + + private final String catalogName; + + private final String databaseName; + + private final String objectName; + + /** + * Constructs an {@link UnresolvedIdentifier} from an array of identifier segments. + * The length of the path must be between 1 (only object name) and 3 (fully qualified + * identifier with catalog, database and object name). + * + * @param path array of identifier segments + * @return an identifier that must be resolved before accessing an object from a {@link CatalogManager} + */ + public static UnresolvedIdentifier of(String... path) { + if (path == null) { + throw new ValidationException("Object identifier can not be null!"); + } + if (path.length < 1 || path.length > 3) { + throw new ValidationException("Object identifier must consist of 1 to 3 parts."); + } + if (Arrays.stream(path).anyMatch(StringUtils::isNullOrWhitespaceOnly)) { + throw new ValidationException("Parts of the object identifier are null or whitespace-only."); + } + + if (path.length == 3) { + return new UnresolvedIdentifier(path[0], path[1], path[2]); + } else if (path.length == 2) { + return new UnresolvedIdentifier(null, path[0], path[1]); + } else { + return new UnresolvedIdentifier(null, null, path[0]); + } + } + + private UnresolvedIdentifier( + @Nullable String catalogName, + @Nullable String databaseName, + String objectName) { + this.catalogName = catalogName; + this.databaseName = databaseName; + this.objectName = Preconditions.checkNotNull(objectName, "Object name must not be null."); + } + + public Optional getCatalogName() { + return Optional.ofNullable(catalogName); + } + + public Optional getDatabaseName() { + return Optional.ofNullable(databaseName); + } + + public String getObjectName() { + return objectName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + UnresolvedIdentifier that = (UnresolvedIdentifier) o; + return catalogName.equals(that.catalogName) && + databaseName.equals(that.databaseName) && + objectName.equals(that.objectName); + } + + @Override + public int hashCode() { + return Objects.hash(catalogName, databaseName, objectName); + } + + @Override + public String toString() { + return Stream.of( + catalogName, + databaseName, + objectName + ).filter(Objects::nonNull) + .map(EncodingUtils::escapeIdentifier) + .collect(Collectors.joining(".")); + } +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java new file mode 100644 index 0000000000000..c1df477167ea0 --- /dev/null +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Parser.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.delegation; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; + +import java.util.List; + +/** + * Provides methods for parsing SQL objects from a SQL string. + */ +@Internal +public interface Parser { + + /** + * Entry point for parsing SQL queries expressed as a String. + * + *

    Note:If the created {@link Operation} is a {@link QueryOperation} + * it must be in a form that will be understood by the + * {@link Planner#translate(List)} method. + * + *

    The produced Operation trees should already be validated. + * + * @param statement the SQL statement to evaluate + * @return parsed queries as trees of relational {@link Operation}s + * @throws org.apache.flink.table.api.SqlParserException when failed to parse the statement + */ + List parse(String statement); + + /** + * Entry point for parsing SQL identifiers expressed as a String. + * + * @param identifier the SQL identifier to parse + * @return parsed identifier + * @throws org.apache.flink.table.api.SqlParserException when failed to parse the identifier + */ + UnresolvedIdentifier parseIdentifier(String identifier); +} diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java index a6d0e8ebb7cea..5bb9266ed1152 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/delegation/Planner.java @@ -29,8 +29,8 @@ /** * This interface serves two purposes: *

      - *
    • SQL parser - transforms a SQL string into a Table API specific tree of - * {@link Operation}s
    • + *
    • SQL parser via {@link #getParser()} - transforms a SQL string into a Table API specific objects + * e.g. tree of {@link Operation}s
    • *
    • relational planner - provides a way to plan, optimize and transform tree of * {@link ModifyOperation} into a runnable form ({@link Transformation})
    • *
    . @@ -43,7 +43,7 @@ * of {@link Planner#translate(List)} will strip any execution configuration from * the DataStream information. * - *

    All Tables referenced in either {@link Planner#parse(String)} or + *

    All Tables referenced in either {@link Parser#parse(String)} or * {@link Planner#translate(List)} should be previously registered in a * {@link org.apache.flink.table.catalog.CatalogManager}, which will be provided during * instantiation of the {@link Planner}. @@ -52,18 +52,11 @@ public interface Planner { /** - * Entry point for parsing sql queries expressed as a String. + * Retrieves a {@link Parser} that provides methods for parsing a SQL string. * - *

    Note:If the created {@link Operation} is a {@link QueryOperation} - * it must be in a form that will be understood by the - * {@link Planner#translate(List)} method. - * - *

    The produced Operation trees should already be validated. - * - * @param statement the sql statement to evaluate - * @return parsed queries as trees of relational {@link Operation}s + * @return initialized {@link Parser} */ - List parse(String statement); + Parser getParser(); /** * Converts a relational tree of {@link ModifyOperation}s into a set of runnable diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java index cd0180117ee3c..faaeae48db43b 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/BatchTableDescriptor.java @@ -19,7 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.Registration; /** * Describes a table connected from a batch environment. @@ -30,8 +30,8 @@ @PublicEvolving public final class BatchTableDescriptor extends ConnectTableDescriptor { - public BatchTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { - super(tableEnv, connectorDescriptor); + public BatchTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { + super(registration, connectorDescriptor); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java index fabc5882e6c72..4ec3f96214071 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/ConnectTableDescriptor.java @@ -20,6 +20,10 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.internal.Registration; +import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; @@ -28,6 +32,7 @@ import javax.annotation.Nullable; import java.util.Collections; +import java.util.HashMap; import java.util.Map; /** @@ -39,13 +44,13 @@ public abstract class ConnectTableDescriptor extends TableDescriptor { - private final TableEnvironment tableEnv; + private final Registration registration; private @Nullable Schema schemaDescriptor; - public ConnectTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { + public ConnectTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { super(connectorDescriptor); - this.tableEnv = tableEnv; + this.registration = registration; } /** @@ -60,37 +65,98 @@ public ConnectTableDescriptor withSchema(Schema schema) { * Searches for the specified table source, configures it accordingly, and registers it as * a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSource(String name) { Preconditions.checkNotNull(name); TableSource tableSource = TableFactoryUtil.findAndCreateTableSource(this); - tableEnv.registerTableSource(name, tableSource); + registration.createTableSource(name, tableSource); } /** * Searches for the specified table sink, configures it accordingly, and registers it as * a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSink(String name) { Preconditions.checkNotNull(name); TableSink tableSink = TableFactoryUtil.findAndCreateTableSink(this); - tableEnv.registerTableSink(name, tableSink); + registration.createTableSink(name, tableSink); } /** * Searches for the specified table source and sink, configures them accordingly, and registers * them as a table under the given name. * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * * @param name table name to be registered in the table environment + * @deprecated use {@link #createTemporaryTable(String)} */ + @Deprecated public void registerTableSourceAndSink(String name) { registerTableSource(name); registerTableSink(name); } + /** + * Registers the table described by underlying properties in a given path. + * + *

    There is no distinction between source and sink at the descriptor level anymore as this + * method does not perform actual class lookup. It only stores the underlying properties. The + * actual source/sink lookup is performed when the table is used. + * + *

    Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will + * be inaccessible in the current session. To make the permanent object available again you can drop the + * corresponding temporary object. + * + *

    NOTE: The schema must be explicitly defined. + * + * @param path path where to register the temporary table + */ + public void createTemporaryTable(String path) { + if (schemaDescriptor == null) { + throw new TableException( + "Table schema must be explicitly defined. To derive schema from the underlying connector" + + " use registerTableSource/registerTableSink/registerTableSourceAndSink."); + } + + Map schemaProperties = schemaDescriptor.toProperties(); + TableSchema tableSchema = getTableSchema(schemaProperties); + + Map properties = new HashMap<>(toProperties()); + schemaProperties.keySet().forEach(properties::remove); + + CatalogTableImpl catalogTable = new CatalogTableImpl( + tableSchema, + properties, + "" + ); + + registration.createTemporaryTable(path, catalogTable); + } + + private TableSchema getTableSchema(Map schemaProperties) { + DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(schemaProperties); + return properties.getTableSchema(Schema.SCHEMA); + } + @Override protected Map additionalProperties() { if (schemaDescriptor != null) { diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java index 8e67959571d0c..65d17171d25ee 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/descriptors/StreamTableDescriptor.java @@ -19,7 +19,7 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.Registration; /** * Describes a table connected from a streaming environment. @@ -30,8 +30,8 @@ @PublicEvolving public final class StreamTableDescriptor extends ConnectTableDescriptor { - public StreamTableDescriptor(TableEnvironment tableEnv, ConnectorDescriptor connectorDescriptor) { - super(tableEnv, connectorDescriptor); + public StreamTableDescriptor(Registration registration, ConnectorDescriptor connectorDescriptor) { + super(registration, connectorDescriptor); } @Override diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java index b5a51d7f1c3c9..83145f084dba2 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogQueryOperation.java @@ -33,16 +33,16 @@ @Internal public class CatalogQueryOperation implements QueryOperation { - private final ObjectIdentifier objectIdentifier; + private final ObjectIdentifier tableIdentifier; private final TableSchema tableSchema; - public CatalogQueryOperation(ObjectIdentifier objectIdentifier, TableSchema tableSchema) { - this.objectIdentifier = objectIdentifier; + public CatalogQueryOperation(ObjectIdentifier tableIdentifier, TableSchema tableSchema) { + this.tableIdentifier = tableIdentifier; this.tableSchema = tableSchema; } - public ObjectIdentifier getObjectIdentifier() { - return objectIdentifier; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } @Override @@ -53,7 +53,7 @@ public TableSchema getTableSchema() { @Override public String asSummaryString() { Map args = new LinkedHashMap<>(); - args.put("identifier", objectIdentifier); + args.put("identifier", tableIdentifier); args.put("fields", tableSchema.getFieldNames()); return OperationUtils.formatWithChildren("CatalogTable", args, getChildren(), Operation::asSummaryString); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java index 1b9e4ab3a3476..86e9a85e0259f 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/CatalogSinkModifyOperation.java @@ -19,11 +19,11 @@ package org.apache.flink.table.operations; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.ObjectIdentifier; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; /** @@ -33,27 +33,28 @@ @Internal public class CatalogSinkModifyOperation implements ModifyOperation { + private final ObjectIdentifier tableIdentifier; private final Map staticPartitions; - private final List tablePath; private final QueryOperation child; private final boolean overwrite; - public CatalogSinkModifyOperation(List tablePath, QueryOperation child) { - this(tablePath, child, new HashMap<>(), false); + public CatalogSinkModifyOperation(ObjectIdentifier tableIdentifier, QueryOperation child) { + this(tableIdentifier, child, new HashMap<>(), false); } - public CatalogSinkModifyOperation(List tablePath, + public CatalogSinkModifyOperation( + ObjectIdentifier tableIdentifier, QueryOperation child, Map staticPartitions, boolean overwrite) { - this.tablePath = tablePath; + this.tableIdentifier = tableIdentifier; this.child = child; this.staticPartitions = staticPartitions; this.overwrite = overwrite; } - public List getTablePath() { - return tablePath; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } public Map getStaticPartitions() { @@ -77,7 +78,7 @@ public T accept(ModifyOperationVisitor visitor) { @Override public String asSummaryString() { Map params = new LinkedHashMap<>(); - params.put("tablePath", tablePath); + params.put("identifier", tableIdentifier); params.put("staticPartitions", staticPartitions); params.put("overwrite", overwrite); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java index 3c806f6b6683d..9491d80532a37 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/CreateTableOperation.java @@ -19,6 +19,7 @@ package org.apache.flink.table.operations.ddl; import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.OperationUtils; @@ -30,14 +31,15 @@ * Operation to describe a CREATE TABLE statement. */ public class CreateTableOperation implements CreateOperation { - private final String[] tablePath; + private final ObjectIdentifier tableIdentifier; private CatalogTable catalogTable; private boolean ignoreIfExists; - public CreateTableOperation(String[] tablePath, + public CreateTableOperation( + ObjectIdentifier tableIdentifier, CatalogTable catalogTable, boolean ignoreIfExists) { - this.tablePath = tablePath; + this.tableIdentifier = tableIdentifier; this.catalogTable = catalogTable; this.ignoreIfExists = ignoreIfExists; } @@ -46,8 +48,8 @@ public CatalogTable getCatalogTable() { return catalogTable; } - public String[] getTablePath() { - return tablePath; + public ObjectIdentifier getTableIdentifier() { + return tableIdentifier; } public boolean isIgnoreIfExists() { @@ -58,7 +60,7 @@ public boolean isIgnoreIfExists() { public String asSummaryString() { Map params = new LinkedHashMap<>(); params.put("catalogTable", catalogTable.toProperties()); - params.put("tablePath", tablePath); + params.put("identifier", tableIdentifier); params.put("ignoreIfExists", ignoreIfExists); return OperationUtils.formatWithChildren( diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java index c9f67db7e3b36..99f15a98a2903 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/ddl/DropTableOperation.java @@ -18,6 +18,7 @@ package org.apache.flink.table.operations.ddl; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.OperationUtils; @@ -29,16 +30,16 @@ * Operation to describe a DROP TABLE statement. */ public class DropTableOperation implements DropOperation { - private final String[] tableName; + private final ObjectIdentifier tableIdentifier; private final boolean ifExists; - public DropTableOperation(String[] tableName, boolean ifExists) { - this.tableName = tableName; + public DropTableOperation(ObjectIdentifier tableIdentifier, boolean ifExists) { + this.tableIdentifier = tableIdentifier; this.ifExists = ifExists; } - public String[] getTableName() { - return this.tableName; + public ObjectIdentifier getTableIdentifier() { + return this.tableIdentifier; } public boolean isIfExists() { @@ -48,7 +49,7 @@ public boolean isIfExists() { @Override public String asSummaryString() { Map params = new LinkedHashMap<>(); - params.put("tableName", tableName); + params.put("identifier", tableIdentifier); params.put("IfExists", ifExists); return OperationUtils.formatWithChildren( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java index 2c98c21bfb61c..017e204d65c71 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/TableEnvironmentTest.java @@ -18,10 +18,10 @@ package org.apache.flink.table.api; -import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.ConnectorCatalogTable; -import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.utils.ConnectorDescriptorMock; import org.apache.flink.table.utils.FormatDescriptorMock; @@ -51,12 +51,15 @@ public void testConnect() throws Exception { .inAppendMode() .registerTableSource("my_table"); - final Catalog catalog = tableEnv.getCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG) + CatalogManager.TableLookupResult lookupResult = tableEnv.catalogManager.getTable(ObjectIdentifier.of( + EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, + EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, + "my_table")) .orElseThrow(AssertionError::new); - final CatalogBaseTable table = catalog - .getTable(new ObjectPath(EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, "my_table")); + assertThat(lookupResult.isTemporary(), equalTo(true)); + CatalogBaseTable table = lookupResult.getTable(); assertThat( table.getSchema(), equalTo( diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java new file mode 100644 index 0000000000000..9dc48de8cd2bd --- /dev/null +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/ParserMock.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.utils; + +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; + +import java.util.List; + +/** + * Mocks {@link Parser} for tests. + */ +public class ParserMock implements Parser { + @Override + public List parse(String statement) { + return null; + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + return null; + } +} diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java index 8dee10c4be615..92f50ff624279 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/PlannerMock.java @@ -19,6 +19,7 @@ package org.apache.flink.table.utils; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; @@ -31,8 +32,8 @@ public class PlannerMock implements Planner { @Override - public List parse(String statement) { - return null; + public Parser getParser() { + return new ParserMock(); } @Override diff --git a/flink-table/flink-table-api-scala-bridge/pom.xml b/flink-table/flink-table-api-scala-bridge/pom.xml index 9a1df4aa7bbe5..f0ae087cc3afa 100644 --- a/flink-table/flink-table-api-scala-bridge/pom.xml +++ b/flink-table/flink-table-api-scala-bridge/pom.xml @@ -52,6 +52,14 @@ under the License. flink-streaming-scala_${scala.binary.version} ${project.version} + + + org.apache.flink + flink-table-api-java + ${project.version} + test-jar + test + diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala index c57b0351e9901..d7bd0ca991503 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala @@ -95,23 +95,48 @@ trait BatchTableEnvironment extends TableEnvironment { def fromDataSet[T](dataSet: DataSet[T], fields: Expression*): Table /** - * Registers the given [[DataSet]] as table in the - * [[TableEnvironment]]'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given [[DataSet]]. + * Registered views can be referenced in SQL queries. * * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]]. * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataSet]] is registered in the catalog. * @param dataSet The [[DataSet]] to register. * @tparam T The type of the [[DataSet]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit /** - * Registers the given [[DataSet]] as table with specified field names in the - * [[TableEnvironment]]'s catalog. + * Creates a view from the given [[DataSet]] in a given path. * Registered tables can be referenced in SQL queries. * + * The field names of the [[Table]] are automatically derived + * from the type of the [[DataSet]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataSet]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataSet The [[DataSet]] out of which to create the view. + * @tparam T The type of the [[DataSet]]. + */ + def createTemporaryView[T](path: String, dataSet: DataSet[T]): Unit + + /** + * Creates a view from the given [[DataSet]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * * Example: * * {{{ @@ -119,13 +144,46 @@ trait BatchTableEnvironment extends TableEnvironment { * tableEnv.registerDataSet("myTable", set, 'a, 'b) * }}} * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataSet]] is registered in the catalog. * @param dataSet The [[DataSet]] to register. * @param fields The field names of the registered table. * @tparam T The type of the [[DataSet]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataSet[T](name: String, dataSet: DataSet[T], fields: Expression*): Unit + /** + * Creates a view from the given [[DataSet]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + * Example: + * + * {{{ + * val set: DataSet[(String, Long)] = ... + * tableEnv.createTemporaryView("cat.db.myTable", set, 'a, 'b) + * }}} + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataSet]] is created. + * See also the [[TableEnvironment]] class description for the format of the + * path. + * @param dataSet The [[DataSet]] out of which to create the view. + * @param fields The field names of the created view. + * @tparam T The type of the [[DataSet]]. + */ + def createTemporaryView[T](path: String, dataSet: DataSet[T], fields: Expression*): Unit + /** * Converts the given [[Table]] into a [[DataSet]] of a specified type. * @@ -193,7 +251,9 @@ trait BatchTableEnvironment extends TableEnvironment { * name of the [[TableSink]] is provided. * @param sinkPathContinued The remaining part of the path of the registered [[TableSink]] to * which the [[Table]] is written. + * @deprecated use `TableEnvironment#insertInto(String, Table)` */ + @deprecated def insertInto( table: Table, queryConfig: BatchQueryConfig, diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index db88251630928..d40b21a0ded01 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -29,7 +29,7 @@ import org.apache.flink.table.functions.{AggregateFunction, TableAggregateFuncti import org.apache.flink.table.sinks.TableSink /** - * This table environment is the entry point and central context for creating Table & SQL + * This table environment is the entry point and central context for creating Table and SQL * API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -113,38 +113,95 @@ trait StreamTableEnvironment extends TableEnvironment { def fromDataStream[T](dataStream: DataStream[T], fields: Expression*): Table /** - * Registers the given [[DataStream]] as table in the - * [[TableEnvironment]]'s catalog. - * Registered tables can be referenced in SQL queries. + * Creates a view from the given [[DataStream]]. + * Registered views can be referenced in SQL queries. * * The field names of the [[Table]] are automatically derived * from the type of the [[DataStream]]. * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataStream]] is registered in the catalog. * @param dataStream The [[DataStream]] to register. * @tparam T The type of the [[DataStream]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit /** - * Registers the given [[DataStream]] as table with specified field names in the - * [[TableEnvironment]]'s catalog. + * Creates a view from the given [[DataStream]] in a given path. * Registered tables can be referenced in SQL queries. * + * The field names of the [[Table]] are automatically derived + * from the type of the [[DataStream]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataStream]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataStream The [[DataStream]] out of which to create the view. + * @tparam T The type of the [[DataStream]]. + */ + def createTemporaryView[T](path: String, dataStream: DataStream[T]): Unit + + /** + * Creates a view from the given [[DataStream]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * * Example: * * {{{ - * val set: DataStream[(String, Long)] = ... - * tableEnv.registerDataStream("myTable", set, 'a, 'b) + * val stream: DataStream[(String, Long)] = ... + * tableEnv.registerDataStream("myTable", stream, 'a, 'b) * }}} * + * The view is registered in the namespace of the current catalog and database. To register the + * view in a different catalog use [[createTemporaryView]]. + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * * @param name The name under which the [[DataStream]] is registered in the catalog. * @param dataStream The [[DataStream]] to register. - * @param fields The field names of the registered table. + * @param fields The field names of the registered view. * @tparam T The type of the [[DataStream]] to register. + * @deprecated use [[createTemporaryView]] */ + @deprecated def registerDataStream[T](name: String, dataStream: DataStream[T], fields: Expression*): Unit + /** + * Creates a view from the given [[DataStream]] in a given path with specified field names. + * Registered views can be referenced in SQL queries. + * + * Example: + * + * {{{ + * val stream: DataStream[(String, Long)] = ... + * tableEnv.createTemporaryView("cat.db.myTable", stream, 'a, 'b) + * }}} + * + * Temporary objects can shadow permanent ones. If a permanent object in a given path exists, + * it will be inaccessible in the current session. To make the permanent object available again + * you can drop the corresponding temporary object. + * + * @param path The path under which the [[DataStream]] is created. + * See also the [[TableEnvironment]] class description for the format of the path. + * @param dataStream The [[DataStream]] out of which to create the view. + * @param fields The field names of the created view. + * @tparam T The type of the [[DataStream]]. + */ + def createTemporaryView[T](path: String, dataStream: DataStream[T], fields: Expression*): Unit + /** * Converts the given [[Table]] into an append [[DataStream]] of a specified type. * @@ -248,7 +305,9 @@ trait StreamTableEnvironment extends TableEnvironment { * of the [[TableSink]] is provided. * @param sinkPathContinued The remaining part of the path of the registered [[TableSink]] to * which the [[Table]] is written. + * @deprecated use `TableEnvironment#insertInto(String, Table)` */ + @deprecated def insertInto( table: Table, queryConfig: StreamQueryConfig, @@ -302,7 +361,7 @@ trait StreamTableEnvironment extends TableEnvironment { * .field("count", "DECIMAL") * .field("proc-time", "TIMESTAMP").proctime()) * .inAppendMode() - * .registerSource("MyTable") + * .createTemporaryTable("MyTable") * }}} * * @param connectorDescriptor connector descriptor describing the external system @@ -313,7 +372,7 @@ trait StreamTableEnvironment extends TableEnvironment { object StreamTableEnvironment { /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -339,7 +398,7 @@ object StreamTableEnvironment { } /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. @@ -367,7 +426,7 @@ object StreamTableEnvironment { } /** - * Creates a table environment that is the entry point and central context for creating Table & + * Creates a table environment that is the entry point and central context for creating Table and * SQL API programs that integrate with the Scala-specific [[DataStream]] API. * * It is unified for bounded and unbounded data processing. diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala index 97da79868eefc..9a1435c3a7840 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImpl.scala @@ -267,6 +267,19 @@ class StreamTableEnvironmentImpl ( Time.milliseconds(queryConfig.getMaxIdleStateRetentionTime)) insertInto(table, sinkPath, sinkPathContinued: _*) } + + override def createTemporaryView[T]( + path: String, + dataStream: DataStream[T]): Unit = { + createTemporaryView(path, fromDataStream(dataStream)) + } + + override def createTemporaryView[T]( + path: String, + dataStream: DataStream[T], + fields: Expression*): Unit = { + createTemporaryView(path, fromDataStream(dataStream, fields: _*)) + } } object StreamTableEnvironmentImpl { diff --git a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala index cf36ed29db083..e238ddc817182 100644 --- a/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala +++ b/flink-table/flink-table-api-scala-bridge/src/test/scala/org/apache/flink/table/api/scala/internal/StreamTableEnvironmentImplTest.scala @@ -23,15 +23,16 @@ import org.apache.flink.api.dag.Transformation import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, _} import org.apache.flink.table.api.TableConfig import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} -import org.apache.flink.table.delegation.{Executor, Planner} -import org.apache.flink.table.operations.{ModifyOperation, Operation} +import org.apache.flink.table.module.ModuleManager +import org.apache.flink.table.operations.ModifyOperation +import org.apache.flink.table.utils.{ExecutorMock, PlannerMock} import org.apache.flink.types.Row + import org.hamcrest.CoreMatchers.equalTo import org.junit.Assert.assertThat import org.junit.Test -import java.util.{Collections, List => JList} -import org.apache.flink.table.module.ModuleManager +import java.util.{Collections, List => JList} /** * Tests for [[StreamTableEnvironmentImpl]]. @@ -91,28 +92,15 @@ class StreamTableEnvironmentImplTest { new TableConfig, env, new TestPlanner(elements.javaStream.getTransformation), - executor, + new ExecutorMock, true) } - private class TestPlanner(transformation: Transformation[_]) extends Planner { - override def parse(statement: String) = throw new AssertionError("Should not be called") - + private class TestPlanner(transformation: Transformation[_]) extends PlannerMock { override def translate(modifyOperations: JList[ModifyOperation]) : JList[Transformation[_]] = { Collections.singletonList(transformation) } - - override def explain(operations: JList[Operation], extended: Boolean) = - throw new AssertionError("Should not be called") - - override def getCompletionHints(statement: String, position: Int) = - throw new AssertionError("Should not be called") } - private val executor = new Executor() { - override def apply(transformations: JList[Transformation[_]]): Unit = {} - - override def execute(jobName: String) = throw new AssertionError("Should not be called") - } } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java new file mode 100644 index 0000000000000..f958003050a29 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/calcite/CalciteParser.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.calcite; + +import org.apache.flink.table.api.SqlParserException; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; + +/** + * Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. + */ +public class CalciteParser { + private final SqlParser.Config config; + + public CalciteParser(SqlParser.Config config) { + this.config = config; + } + + /** + * Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. + * + * @param sql a sql string to parse + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the statement + */ + public SqlNode parse(String sql) { + try { + SqlParser parser = SqlParser.create(sql, config); + return parser.parseStmt(); + } catch (SqlParseException e) { + throw new SqlParserException("SQL parse failed. " + e.getMessage()); + } + } + + /** + * Parses a SQL string as an identifier into a {@link SqlIdentifier}. + * + * @param identifier a sql string to parse as an identifier + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the identifier + */ + public SqlIdentifier parseIdentifier(String identifier) { + try { + SqlParser parser = SqlParser.create(identifier, config); + SqlNode sqlNode = parser.parseExpression(); + return (SqlIdentifier) sqlNode; + } catch (Exception e) { + throw new SqlParserException(String.format( + "Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); + } + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java index d53a40261f141..f869f46d6b433 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogCalciteSchema.java @@ -19,7 +19,7 @@ package org.apache.flink.table.planner.catalog; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.schema.Schema; @@ -39,13 +39,13 @@ public class CatalogCalciteSchema extends FlinkSchema { private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; // Flag that tells if the current planner should work in a batch or streaming mode. private final boolean isStreamingMode; - public CatalogCalciteSchema(String catalogName, Catalog catalog, boolean isStreamingMode) { + public CatalogCalciteSchema(String catalogName, CatalogManager catalog, boolean isStreamingMode) { this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalog; this.isStreamingMode = isStreamingMode; } @@ -57,8 +57,8 @@ public CatalogCalciteSchema(String catalogName, Catalog catalog, boolean isStrea */ @Override public Schema getSubSchema(String schemaName) { - if (catalog.databaseExists(schemaName)) { - return new DatabaseCalciteSchema(schemaName, catalogName, catalog, isStreamingMode); + if (catalogManager.schemaExists(catalogName, schemaName)) { + return new DatabaseCalciteSchema(schemaName, catalogName, catalogManager, isStreamingMode); } else { return null; } @@ -66,7 +66,7 @@ public Schema getSubSchema(String schemaName) { @Override public Set getSubSchemaNames() { - return new HashSet<>(catalog.listDatabases()); + return catalogManager.listSchemas(catalogName); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java index 7dd5da3b8e0d9..0afd2a89d5ee2 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogManagerCalciteSchema.java @@ -62,14 +62,16 @@ public Set getTableNames() { @Override public Schema getSubSchema(String name) { - return catalogManager.getCatalog(name) - .map(catalog -> new CatalogCalciteSchema(name, catalog, isStreamingMode)) - .orElse(null); + if (catalogManager.schemaExists(name)) { + return new CatalogCalciteSchema(name, catalogManager, isStreamingMode); + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return new HashSet<>(catalogManager.getCatalogs()); + return new HashSet<>(catalogManager.listCatalogs()); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java index ade4b46d6848c..233c190f7ec66 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/DatabaseCalciteSchema.java @@ -21,12 +21,12 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ConnectorCatalogTable; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.QueryOperationCatalogView; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; @@ -50,6 +50,8 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.HashSet; import java.util.List; @@ -66,109 +68,154 @@ class DatabaseCalciteSchema extends FlinkSchema { private final String databaseName; private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; // Flag that tells if the current planner should work in a batch or streaming mode. private final boolean isStreamingMode; - public DatabaseCalciteSchema(String databaseName, String catalogName, Catalog catalog, boolean isStreamingMode) { + public DatabaseCalciteSchema(String databaseName, String catalogName, CatalogManager catalog, boolean isStreamingMode) { this.databaseName = databaseName; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalog; this.isStreamingMode = isStreamingMode; } @Override public Table getTable(String tableName) { - ObjectPath tablePath = new ObjectPath(databaseName, tableName); + ObjectIdentifier identifier = ObjectIdentifier.of(catalogName, databaseName, 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) + ); + } + }) + .orElse(null); + } - try { - if (!catalog.tableExists(tablePath)) { - return null; + private Table convertPermanentTable( + ObjectPath tablePath, + CatalogBaseTable table, + @Nullable TableFactory tableFactory) { + // TODO supports GenericCatalogView + if (table instanceof QueryOperationCatalogView) { + return convertQueryOperationView(tablePath, (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) { + return convertCatalogTable(tablePath, (CatalogTable) table, tableFactory); + } else { + throw new TableException("Unsupported table type: " + table); + } + } - CatalogBaseTable table = catalog.getTable(tablePath); - - // TODO supports GenericCatalogView - if (table instanceof QueryOperationCatalogView) { - QueryOperationCatalogView view = (QueryOperationCatalogView) table; - QueryOperation operation = view.getQueryOperation(); - if (operation instanceof DataStreamQueryOperation) { - List qualifiedName = Arrays.asList(catalogName, databaseName, tableName); - ((DataStreamQueryOperation) operation).setQualifiedName(qualifiedName); - } else if (operation instanceof RichTableSourceQueryOperation) { - List qualifiedName = Arrays.asList(catalogName, databaseName, tableName); - ((RichTableSourceQueryOperation) operation).setQualifiedName(qualifiedName); - } - return QueryOperationCatalogViewTable.createCalciteTable(view); - } else if (table instanceof ConnectorCatalogTable) { - return convertConnectorTable((ConnectorCatalogTable) table, tablePath); - } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table); + private Table convertTemporaryTable( + ObjectPath tablePath, + CatalogBaseTable table) { + // TODO supports GenericCatalogView + if (table instanceof QueryOperationCatalogView) { + return convertQueryOperationView(tablePath, (QueryOperationCatalogView) table); + } else if (table instanceof ConnectorCatalogTable) { + ConnectorCatalogTable connectorTable = (ConnectorCatalogTable) table; + if ((connectorTable).getTableSource().isPresent()) { + return convertSourceTable(connectorTable, TableStats.UNKNOWN); } else { - throw new TableException("Unsupported table type: " + table); + return convertSinkTable(connectorTable); } - } catch (TableNotExistException | CatalogException e) { - // TableNotExistException should never happen, because we are checking it exists - // via catalog.tableExists - throw new TableException(format( - "A failure occurred when accessing table. Table path [%s, %s, %s]", - catalogName, - databaseName, - tableName), e); + } else if (table instanceof CatalogTable) { + return convertCatalogTable(tablePath, (CatalogTable) table, null); + } else { + throw new TableException("Unsupported table type: " + table); + } + } + + private Table convertQueryOperationView(ObjectPath tablePath, QueryOperationCatalogView table) { + QueryOperation operation = table.getQueryOperation(); + if (operation instanceof DataStreamQueryOperation) { + List qualifiedName = Arrays.asList(catalogName, databaseName, tablePath.getObjectName()); + ((DataStreamQueryOperation) operation).setQualifiedName(qualifiedName); + } else if (operation instanceof RichTableSourceQueryOperation) { + List qualifiedName = Arrays.asList(catalogName, databaseName, tablePath.getObjectName()); + ((RichTableSourceQueryOperation) operation).setQualifiedName(qualifiedName); + } + return QueryOperationCatalogViewTable.createCalciteTable(table); + } + + private Table convertSinkTable(ConnectorCatalogTable table) { + Optional tableSinkTable = table.getTableSink() + .map(tableSink -> new TableSinkTable<>( + tableSink, + FlinkStatistic.UNKNOWN())); + if (tableSinkTable.isPresent()) { + return tableSinkTable.get(); + } else { + throw new TableException("Cannot convert a connector table " + + "without either source or sink."); } } - private Table convertConnectorTable( + private Table convertSourceTable( ConnectorCatalogTable table, - ObjectPath tablePath) throws TableNotExistException { - if (table.getTableSource().isPresent()) { - 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."); - } + 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."); + } + + return new TableSourceTable<>( + tableSource, + isStreamingMode, + FlinkStatistic.builder().tableStats(tableStats).build(), + null); + } - TableStats tableStats = TableStats.UNKNOWN; + private TableStats extractTableStats(ConnectorCatalogTable table, ObjectPath tablePath) { + TableStats tableStats = TableStats.UNKNOWN; + try { // TODO supports stats for partitionable table if (!table.isPartitioned()) { + Catalog catalog = catalogManager.getCatalog(catalogName).get(); CatalogTableStatistics tableStatistics = catalog.getTableStatistics(tablePath); CatalogColumnStatistics columnStatistics = catalog.getTableColumnStatistics(tablePath); tableStats = convertToTableStats(tableStatistics, columnStatistics); } - return new TableSourceTable<>( - tableSource, - isStreamingMode, - FlinkStatistic.builder().tableStats(tableStats).build(), - null); - } else { - Optional tableSinkTable = table.getTableSink() - .map(tableSink -> new TableSinkTable<>( - tableSink, - FlinkStatistic.UNKNOWN())); - if (tableSinkTable.isPresent()) { - return tableSinkTable.get(); - } else { - throw new TableException("Cannot convert a connector table " + - "without either source or sink."); - } + return tableStats; + } catch (TableNotExistException e) { + throw new TableException(format( + "Could not access table partitions for table: [%s, %s, %s]", + catalogName, + databaseName, + tablePath.getObjectName()), e); } } - private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { + private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nullable TableFactory tableFactory) { TableSource tableSource; - Optional tableFactory = catalog.getTableFactory(); - if (tableFactory.isPresent()) { - TableFactory tf = tableFactory.get(); - if (tf instanceof TableSourceFactory) { - tableSource = ((TableSourceFactory) tf).createTableSource(tablePath, table); + if (tableFactory != null) { + if (tableFactory instanceof TableSourceFactory) { + tableSource = ((TableSourceFactory) tableFactory).createTableSource(tablePath, table); } else { - throw new TableException(String.format("Cannot query a sink-only table. TableFactory provided by catalog %s must implement TableSourceFactory", - catalog.getClass())); + throw new TableException( + "Cannot query a sink-only table. TableFactory provided by catalog must implement TableSourceFactory"); } } else { tableSource = TableFactoryUtil.findAndCreateTableSource(table); @@ -188,11 +235,7 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { @Override public Set getTableNames() { - try { - return new HashSet<>(catalog.listTables(databaseName)); - } catch (DatabaseNotExistException e) { - throw new CatalogException(e); - } + return catalogManager.listTables(catalogName, databaseName); } @Override diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java new file mode 100644 index 0000000000000..878cb761d1354 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/ParserImpl.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.delegation; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.calcite.CalciteParser; +import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; +import org.apache.flink.table.planner.operations.SqlToOperationConverter; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; + +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +/** + * Implementation of {@link Parser} that uses Calcite. + */ +public class ParserImpl implements Parser { + + private final CatalogManager catalogManager; + + // we use supplier pattern here in order to use the most up to + // date configuration. Users might change the parser configuration in a TableConfig in between + // multiple statements parsing + private final Supplier validatorSupplier; + private final Supplier calciteParserSupplier; + + public ParserImpl( + CatalogManager catalogManager, + Supplier validatorSupplier, + Supplier calciteParserSupplier) { + this.catalogManager = catalogManager; + this.validatorSupplier = validatorSupplier; + this.calciteParserSupplier = calciteParserSupplier; + } + + @Override + public List parse(String statement) { + CalciteParser parser = calciteParserSupplier.get(); + FlinkPlannerImpl planner = validatorSupplier.get(); + // parse the sql query + SqlNode parsed = parser.parse(statement); + + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, parsed) + .orElseThrow(() -> new TableException("Unsupported query: " + statement)); + return Collections.singletonList(operation); + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + CalciteParser parser = calciteParserSupplier.get(); + SqlIdentifier sqlIdentifier = parser.parseIdentifier(identifier); + return UnresolvedIdentifier.of(sqlIdentifier.names.toArray(new String[0])); + } +} diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index df093e1e2076d..43cde6d12a97d 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -27,6 +27,7 @@ import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.planner.calcite.CalciteConfig; import org.apache.flink.table.planner.calcite.CalciteConfig$; +import org.apache.flink.table.planner.calcite.CalciteParser; import org.apache.flink.table.planner.calcite.FlinkContextImpl; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -150,6 +151,15 @@ public FlinkPlannerImpl createFlinkPlanner(String currentCatalog, String current cluster); } + /** + * Creates a configured instance of {@link CalciteParser}. + * + * @return configured calcite parser + */ + public CalciteParser createCalciteParser() { + return new CalciteParser(getSqlParserConfig()); + } + private FlinkCalciteCatalogReader createCatalogReader( boolean lenientCaseSensitivity, String currentCatalog, diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java index d19c725aa590a..596aa492e5a99 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/operations/SqlToOperationConverter.java @@ -25,8 +25,11 @@ import org.apache.flink.sql.parser.dml.RichSqlInsert; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; @@ -45,6 +48,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -60,11 +64,15 @@ */ public class SqlToOperationConverter { private FlinkPlannerImpl flinkPlanner; + private CatalogManager catalogManager; //~ Constructors ----------------------------------------------------------- - private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { + private SqlToOperationConverter( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager) { this.flinkPlanner = flinkPlanner; + this.catalogManager = catalogManager; } /** @@ -72,24 +80,27 @@ private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { * SqlNode will have it's implementation in the #convert(type) method whose 'type' argument * is subclass of {@code SqlNode}. * - * @param flinkPlanner FlinkPlannerImpl to convertCreateTable sql node to rel node - * @param sqlNode SqlNode to execute on + * @param flinkPlanner FlinkPlannerImpl to convertCreateTable sql node to rel node + * @param catalogManager CatalogManager to resolve full path for operations + * @param sqlNode SqlNode to execute on */ - public static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) { + public static Optional convert( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager, + SqlNode sqlNode) { // validate the query final SqlNode validated = flinkPlanner.validate(sqlNode); - SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner); + SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { - return converter.convertCreateTable((SqlCreateTable) validated); - } if (validated instanceof SqlDropTable) { - return converter.convertDropTable((SqlDropTable) validated); + return Optional.of(converter.convertCreateTable((SqlCreateTable) validated)); + } else if (validated instanceof SqlDropTable) { + return Optional.of(converter.convertDropTable((SqlDropTable) validated)); } else if (validated instanceof RichSqlInsert) { - return converter.convertSqlInsert((RichSqlInsert) validated); + return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated)); } else if (validated.getKind().belongsTo(SqlKind.QUERY)) { - return converter.convertSqlQuery(validated); + return Optional.of(converter.convertSqlQuery(validated)); } else { - throw new TableException("Unsupported node type " - + validated.getClass().getSimpleName()); + return Optional.empty(); } } @@ -128,23 +139,42 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { partitionKeys, properties, tableComment); - return new CreateTableOperation(sqlCreateTable.fullTableName(), catalogTable, + + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlCreateTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + return new CreateTableOperation( + identifier, + catalogTable, sqlCreateTable.isIfNotExists()); } /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - return new DropTableOperation(sqlDropTable.fullTableName(), sqlDropTable.getIfExists()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlDropTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } /** Convert insert into statement. */ private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; + + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + PlannerQueryOperation query = (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, + insert.getSource()) + .orElseThrow(() -> new TableException( + "Unsupported node type " + insert.getSource().getClass().getSimpleName())); + return new CatalogSinkModifyOperation( - targetTablePath, - (PlannerQueryOperation) SqlToOperationConverter.convert(flinkPlanner, - insert.getSource()), + identifier, + query, insert.getStaticPartitionKVs(), insert.isOverwrite()); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index a6720eed80799..ac0d3b68957e3 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -297,7 +297,7 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { @Override public RelNode visit(CatalogQueryOperation catalogTable) { - ObjectIdentifier objectIdentifier = catalogTable.getObjectIdentifier(); + ObjectIdentifier objectIdentifier = catalogTable.getTableIdentifier(); return relBuilder.scan( objectIdentifier.getCatalogName(), objectIdentifier.getDatabaseName(), diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index f85062fd4e3c4..6e6d45ae0ade4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -19,9 +19,8 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.sql.parser.ExtendedSqlNode -import org.apache.flink.table.api.{SqlParserException, TableException, ValidationException} +import org.apache.flink.table.api.{TableException, ValidationException} -import com.google.common.collect.ImmutableList import org.apache.calcite.config.NullCollation import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ @@ -29,7 +28,6 @@ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} -import org.apache.calcite.sql.parser.{SqlParser, SqlParseException => CSqlParseException} import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} @@ -54,24 +52,13 @@ class FlinkPlannerImpl( cluster: RelOptCluster) { val operatorTable: SqlOperatorTable = config.getOperatorTable - /** Holds the trait definitions to be registered with planner. May be null. */ - val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs - val parserConfig: SqlParser.Config = config.getParserConfig + val parser: CalciteParser = new CalciteParser(config.getParserConfig) val convertletTable: SqlRexConvertletTable = config.getConvertletTable val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ - private def ready() { - if (this.traitDefs != null) { - cluster.getPlanner.clearRelTraitDefs() - for (traitDef <- this.traitDefs) { - cluster.getPlanner.addRelTraitDef(traitDef) - } - } - } - def getCompletionHints(sql: String, cursor: Int): Array[String] = { val advisorValidator = new SqlAdvisorValidator( operatorTable, @@ -109,18 +96,6 @@ class FlinkPlannerImpl( validator } - def parse(sql: String): SqlNode = { - try { - ready() - val parser: SqlParser = SqlParser.create(sql, parserConfig) - val sqlNode: SqlNode = parser.parseStmt - sqlNode - } catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } - } - def validate(sqlNode: SqlNode): SqlNode = { val catalogReader = catalogReaderSupplier.apply(false) // do pre-validate rewrite. @@ -179,15 +154,7 @@ class FlinkPlannerImpl( schemaPath: util.List[String], viewPath: util.List[String]): RelRoot = { - val parser: SqlParser = SqlParser.create(queryString, parserConfig) - var sqlNode: SqlNode = null - try { - sqlNode = parser.parseQuery - } - catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } + val sqlNode: SqlNode = parser.parse(queryString) val catalogReader: CalciteCatalogReader = catalogReaderSupplier.apply(false) .withSchemaPath(schemaPath) val validator: SqlValidator = diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala index ac9670e76b218..691910b476928 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala @@ -21,19 +21,17 @@ package org.apache.flink.table.planner.delegation import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.dag.Transformation import org.apache.flink.configuration.Configuration -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.{TableConfig, TableEnvironment, TableException} import org.apache.flink.table.catalog._ -import org.apache.flink.table.delegation.{Executor, Planner} +import org.apache.flink.table.delegation.{Executor, Parser, Planner} import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} import org.apache.flink.table.operations.OutputConversionModifyOperation.UpdateMode import org.apache.flink.table.operations._ -import org.apache.flink.table.planner.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} +import org.apache.flink.table.planner.calcite.{CalciteParser, FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema import org.apache.flink.table.planner.expressions.PlannerTypeInferenceUtilImpl -import org.apache.flink.table.planner.operations.SqlToOperationConverter import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSink import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.physical.FlinkPhysicalRel @@ -48,10 +46,10 @@ import org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.plan.{RelTrait, RelTraitDef} import org.apache.calcite.rel.RelNode -import org.apache.calcite.sql.SqlKind import org.apache.calcite.tools.FrameworkConfig import java.util +import java.util.function.{Supplier => JSupplier} import _root_.scala.collection.JavaConversions._ @@ -82,7 +80,8 @@ abstract class PlannerBase( executor.asInstanceOf[ExecutorBase].setTableConfig(config) - private val plannerContext: PlannerContext = + @VisibleForTesting + private[flink] val plannerContext: PlannerContext = new PlannerContext( config, functionCatalog, @@ -90,6 +89,19 @@ abstract class PlannerBase( getTraitDefs.toList ) + private val parser: Parser = new ParserImpl( + catalogManager, + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = createFlinkPlanner + }, + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + new JSupplier[CalciteParser] { + override def get(): CalciteParser = plannerContext.createCalciteParser() + } + ) + /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */ private[flink] def getRelBuilder: FlinkRelBuilder = { val currentCatalogName = catalogManager.getCurrentCatalog @@ -120,21 +132,7 @@ abstract class PlannerBase( executor.asInstanceOf[ExecutorBase].getExecutionEnvironment } - override def parse(stmt: String): util.List[Operation] = { - val planner = createFlinkPlanner - // parse the sql query - val parsed = planner.parse(stmt) - parsed match { - case insert: RichSqlInsert => - List(SqlToOperationConverter.convert(planner, insert)) - case query if query.getKind.belongsTo(SqlKind.QUERY) => - List(SqlToOperationConverter.convert(planner, query)) - case ddl if ddl.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, ddl)) - case _ => - throw new TableException(s"Unsupported query: $stmt") - } - } + override def getParser: Parser = parser override def translate( modifyOperations: util.List[ModifyOperation]): util.List[Transformation[_]] = { @@ -174,7 +172,7 @@ abstract class PlannerBase( case catalogSink: CatalogSinkModifyOperation => val input = getRelBuilder.queryOperation(modifyOperation.getChild).build() - val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) + val identifier = catalogSink.getTableIdentifier getTableSink(identifier).map { case (table, sink) => TableSinkUtils.validateSink(catalogSink, identifier, sink, table.getPartitionKeys) sink match { @@ -190,10 +188,11 @@ abstract class PlannerBase( s"${classOf[OverwritableTableSink].getSimpleName} but actually got " + sink.getClass.getName) } - LogicalSink.create(input, sink, catalogSink.getTablePath.mkString("."), table) + LogicalSink.create(input, sink, identifier.toString, table) } match { case Some(sinkRel) => sinkRel - case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") + case None => + throw new TableException(s"Sink ${catalogSink.getTableIdentifier} does not exists") } case outputConversion: OutputConversionModifyOperation => @@ -252,9 +251,10 @@ abstract class PlannerBase( */ protected def translateToPlan(execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] - private def getTableSink( - tableIdentifier: ObjectIdentifier): Option[(CatalogTable, TableSink[_])] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(tableIdentifier)) match { + private def getTableSink(objectIdentifier: ObjectIdentifier) + : Option[(CatalogTable, TableSink[_])] = { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => val table = s.asInstanceOf[ConnectorCatalogTable[_, _]] JavaScalaConversionUtil.toScala(table.getTableSink) match { @@ -263,10 +263,10 @@ abstract class PlannerBase( } case Some(s) if s.isInstanceOf[CatalogTable] => - val catalog = catalogManager.getCatalog(tableIdentifier.getCatalogName) + val catalog = catalogManager.getCatalog(objectIdentifier.getCatalogName) val table = s.asInstanceOf[CatalogTable] if (catalog.isPresent && catalog.get().getTableFactory.isPresent) { - val objectPath = tableIdentifier.toObjectPath + val objectPath = objectIdentifier.toObjectPath val sink = TableFactoryUtil.createTableSinkForCatalogTable( catalog.get(), table, diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java index 7bd663235ab63..c3d2317972f8b 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogStatisticsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.ConnectorCatalogTable; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; @@ -61,10 +62,16 @@ public class CatalogStatisticsTest { public void testGetStatsFromCatalog() throws Exception { EnvironmentSettings settings = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build(); TableEnvironment tEnv = TableEnvironment.create(settings); - tEnv.registerTableSource("T1", new TestTableSource(true, tableSchema)); - tEnv.registerTableSource("T2", new TestTableSource(true, tableSchema)); Catalog catalog = tEnv.getCatalog(tEnv.getCurrentCatalog()).orElse(null); assertNotNull(catalog); + catalog.createTable( + ObjectPath.fromString("default_database.T1"), + ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), + false); + catalog.createTable( + ObjectPath.fromString("default_database.T2"), + ConnectorCatalogTable.source(new TestTableSource(true, tableSchema), true), + false); catalog.alterTableStatistics(ObjectPath.fromString("default_database.T1"), new CatalogTableStatistics(100, 10, 1000L, 2000L), true); diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 9f872752b066a..5c11d89725d97 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -37,6 +37,7 @@ import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.planner.calcite.CalciteParser; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.catalog.CatalogManagerCalciteSchema; import org.apache.flink.table.planner.delegation.PlannerContext; @@ -119,7 +120,8 @@ public void testCreateTable() { " 'kafka.topic' = 'log.test'\n" + ")\n"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - Operation operation = parse(sql, planner); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + Operation operation = parse(sql, planner, parser); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -137,6 +139,7 @@ public void testCreateTable() { @Test(expected = SqlConversionException.class) public void testCreateTableWithPkUniqueKeys() { FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); final String sql = "CREATE TABLE tbl1 (\n" + " a bigint,\n" + " b varchar, \n" + @@ -150,7 +153,7 @@ public void testCreateTableWithPkUniqueKeys() { " 'connector' = 'kafka', \n" + " 'kafka.topic' = 'log.test'\n" + ")\n"; - parse(sql, planner); + parse(sql, planner, parser); } @Test @@ -165,9 +168,10 @@ public void testCreateTableWithMinusInOptionKey() { " 'a.b-c-d.e-f1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad')\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -187,7 +191,8 @@ public void testCreateTableWithMinusInOptionKey() { public void testSqlInsertWithStaticPartition() { final String sql = "insert into t1 partition(a=1) select b, c, d from t2"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - Operation operation = parse(sql, planner); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.HIVE); + Operation operation = parse(sql, planner, parser); assert operation instanceof CatalogSinkModifyOperation; CatalogSinkModifyOperation sinkModifyOperation = (CatalogSinkModifyOperation) operation; final Map expectedStaticPartitions = new HashMap<>(); @@ -323,9 +328,10 @@ public void testCreateTableWithFullDataTypes() { } final String sql = buffer.toString(); final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + final CalciteParser parser = getParserBySqlDialect(SqlDialect.DEFAULT); + SqlNode node = parser.parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); @@ -345,9 +351,9 @@ private static TestItem createTestItem(Object... args) { return testItem; } - private Operation parse(String sql, FlinkPlannerImpl planner) { - SqlNode node = planner.parse(sql); - return SqlToOperationConverter.convert(planner, node); + private Operation parse(String sql, FlinkPlannerImpl planner, CalciteParser parser) { + SqlNode node = parser.parse(sql); + return SqlToOperationConverter.convert(planner, catalogManager, node).get(); } private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { @@ -356,6 +362,11 @@ private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { catalogManager.getCurrentDatabase()); } + private CalciteParser getParserBySqlDialect(SqlDialect sqlDialect) { + tableConfig.setSqlDialect(sqlDialect); + return plannerContext.createCalciteParser(); + } + //~ Inner Classes ---------------------------------------------------------- private static class TestItem { diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml index 06d8b8812b194..0b24954dc9dd4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml @@ -283,14 +283,14 @@ Calc(select=[a, b, c, e, f]): rowcount = , cumulative cost = {rows, cpu, io, net ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[sink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -304,11 +304,11 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS c +- Calc(select=[a]) +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) -Sink(name=[sink1], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[>(cnt, 10)]) +- Reused(reference_id=[1]) -Sink(name=[sink2], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[<(cnt, 10)]) +- Reused(reference_id=[1]) @@ -362,14 +362,14 @@ Sink(name=[sink2], fields=[a, cnt]) ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[sink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -383,11 +383,11 @@ HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS c +- Calc(select=[a]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -Sink(name=[sink1], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[>(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) -Sink(name=[sink2], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[<(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) @@ -441,13 +441,13 @@ Sink(name=[sink2], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[sink], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[>(a, 10)]) +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) @@ -477,13 +477,13 @@ Sink(name=[sink], fields=[a, b, c]) ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[sink], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`sink`], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, b, c], where=[>(a, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- BoundedStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml index 62ac33d4f7994..46661c62c59f8 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml @@ -345,14 +345,14 @@ Calc(select=[a, b, c, e, f], updateAsRetraction=[false], accMode=[Acc]): rowcoun ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[upsertSink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -365,11 +365,11 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], reuse_id=[1]) +- Calc(select=[a]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) -Sink(name=[upsertSink1], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[>(cnt, 10)]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[a, cnt]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- Calc(select=[a, cnt], where=[<(cnt, 10)]) +- Reused(reference_id=[1]) @@ -427,14 +427,14 @@ Sink(name=[upsertSink2], fields=[a, cnt]) ($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) +- LogicalProject(a=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) -LogicalSink(name=[upsertSink2], fields=[a, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt]) +- LogicalProject(a=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($1, 10)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT()]) @@ -447,11 +447,11 @@ GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[fal +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} -Sink(name=[upsertSink1], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[>(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, cnt], where=[<(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Reused(reference_id=[1]) @@ -509,13 +509,13 @@ Sink(name=[upsertSink2], fields=[a, cnt], updateAsRetraction=[false], accMode=[A ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[appendSink], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c]) +- Calc(select=[a, b, c], where=[>(a, 10)]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c]) @@ -545,13 +545,13 @@ Sink(name=[appendSink], fields=[a, b, c]) ($0, 10)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) == Optimized Logical Plan == -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- Calc(select=[a, b, c], where=[>(a, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} +- DataStreamScan(table=[[default_catalog, default_database, MyTable1]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory} @@ -635,7 +635,7 @@ SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], updateAsRetraction=[false], ac =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -62,7 +62,7 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[sink1], fields=[a, b, c]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, b, c]) +- Reused(reference_id=[1]) HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_COUNT(count$0) AS cnt], reuse_id=[3]) @@ -74,11 +74,11 @@ HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_COUNT(count$0) AS cn +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')]) +- Reused(reference_id=[2]) -Sink(name=[sink2], fields=[b, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[b, cnt]) +- Calc(select=[b, cnt], where=[<(b, 4)]) +- Reused(reference_id=[3]) -Sink(name=[sink3], fields=[b, cnt]) +Sink(name=[`default_catalog`.`default_database`.`sink3`], fields=[b, cnt]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))]) +- Reused(reference_id=[3]) ]]> @@ -87,7 +87,7 @@ Sink(name=[sink3], fields=[b, cnt]) @@ -187,7 +187,7 @@ Sink(name=[sink2], fields=[a, b1]) =($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[sink2], fields=[a1, b, c1]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -289,10 +289,10 @@ Calc(select=[a AS a1, b, c AS c2], reuse_id=[3]) +- Calc(select=[a AS a2, c], where=[>=(b, 5)]) +- Reused(reference_id=[2]) -Sink(name=[sink1], fields=[a1, b, c2]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a1, b, c2]) +- Reused(reference_id=[3]) -Sink(name=[sink2], fields=[a1, b, c1]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a1, b, c1]) +- Calc(select=[a1, b, c1]) +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[left]) :- Exchange(distribution=[hash[a3]]) @@ -307,13 +307,13 @@ Sink(name=[sink2], fields=[a1, b, c1]) @@ -456,14 +456,14 @@ Sink(name=[sink3], fields=[a]) @@ -615,7 +615,7 @@ Sink(name=[sink2], fields=[a, sum_c, time]) ($0, 50)]) +- LogicalUnion(all=[true]) @@ -633,7 +633,7 @@ LogicalSink(name=[sink1], fields=[a, total_c]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[sink2], fields=[a, total_c]) +LogicalSink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalUnion(all=[true]) @@ -671,14 +671,14 @@ Calc(select=[a, total_c], reuse_id=[2]) Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], reuse_id=[3]) +- Reused(reference_id=[1]) -Sink(name=[sink1], fields=[a, total_c]) +Sink(name=[`default_catalog`.`default_database`.`sink1`], fields=[a, total_c]) +- Union(all=[true], union=[a, total_c]) :- Calc(select=[a, total_c], where=[>(a, 50)]) : +- Reused(reference_id=[2]) +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)]) +- Reused(reference_id=[3]) -Sink(name=[sink2], fields=[a, total_c]) +Sink(name=[`default_catalog`.`default_database`.`sink2`], fields=[a, total_c]) +- Union(all=[true], union=[a, total_c]) :- Calc(select=[a, total_c], where=[<(a, 50)]) : +- Reused(reference_id=[2]) @@ -690,7 +690,7 @@ Sink(name=[sink2], fields=[a, total_c]) =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -62,7 +62,7 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[1]) GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[3]) @@ -73,11 +73,11 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[2]) -Sink(name=[retractSink], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[3]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[3]) ]]> @@ -86,7 +86,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac @@ -125,7 +125,7 @@ Sink(name=[appendSink2], fields=[total_min], updateAsRetraction=[false], accMode @@ -181,7 +181,7 @@ Sink(name=[appendSink2], fields=[a, b1], updateAsRetraction=[false], accMode=[Ac =($0, 0)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[appendSink2], fields=[a1, b, c1]) +LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1]) +- LogicalProject(a1=[$2], b=[$3], c1=[$1]) +- LogicalFilter(condition=[=($2, $0)]) +- LogicalJoin(condition=[true], joinType=[inner]) @@ -282,10 +282,10 @@ Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc], reuse_id +- Calc(select=[a AS a2, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[2]) -Sink(name=[appendSink1], fields=[a1, b, c2], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a1, b, c2], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[appendSink2], fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +- Calc(select=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc]) +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], updateAsRetraction=[false], accMode=[Acc]) :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc]) @@ -299,13 +299,13 @@ Sink(name=[appendSink2], fields=[a1, b, c1], updateAsRetraction=[false], accMode @@ -331,7 +331,7 @@ Sink(name=[appendSink2], fields=[total_min], updateAsRetraction=[false], accMode ($0, 50)]) +- LogicalUnion(all=[true]) @@ -349,7 +349,7 @@ LogicalSink(name=[appendSink1], fields=[a, total_c]) +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[appendSink2], fields=[a, total_c]) +LogicalSink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, total_c]) +- LogicalProject(a=[$0], total_c=[$1]) +- LogicalFilter(condition=[<($0, 50)]) +- LogicalUnion(all=[true]) @@ -383,14 +383,14 @@ Calc(select=[a, total_c], updateAsRetraction=[false], accMode=[Acc], reuse_id=[2 Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], updateAsRetraction=[false], accMode=[Acc], reuse_id=[3]) +- Reused(reference_id=[1]) -Sink(name=[appendSink1], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink1`], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) :- Calc(select=[a, total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc]) : +- Reused(reference_id=[2]) +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[appendSink2], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink2`], fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) +- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc]) :- Calc(select=[a, total_c], where=[<(a, 50)], updateAsRetraction=[false], accMode=[Acc]) : +- Reused(reference_id=[2]) @@ -402,7 +402,7 @@ Sink(name=[appendSink2], fields=[a, total_c], updateAsRetraction=[false], accMod @@ -444,7 +444,7 @@ Sink(name=[retractSink], fields=[total_min], updateAsRetraction=[true], accMode= @@ -508,14 +508,14 @@ Sink(name=[appendSink3], fields=[a]) @@ -573,7 +573,7 @@ Sink(name=[upsertSink], fields=[total_min], updateAsRetraction=[false], accMode= @@ -624,12 +624,12 @@ Sink(name=[retractSink], fields=[total_min], updateAsRetraction=[true], accMode= =($1, 5)]) +- LogicalUnion(all=[true]) @@ -642,7 +642,7 @@ LogicalSink(name=[retractSink1], fields=[a, b, c]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[retractSink2], fields=[b, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($0, 4)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -657,7 +657,7 @@ LogicalSink(name=[retractSink2], fields=[b, cnt]) +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[upsertSink], fields=[b, cnt]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt]) +- LogicalProject(b=[$0], cnt=[$1]) +- LogicalFilter(condition=[<($0, 4)]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -680,13 +680,13 @@ TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [Tes Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1]) +- Reused(reference_id=[2]) -Sink(name=[appendSink], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`appendSink`], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) +- Reused(reference_id=[1]) Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[3]) +- Reused(reference_id=[2]) -Sink(name=[retractSink1], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`retractSink1`], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) +- Union(all=[true], union=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) :- Calc(select=[a, b, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc]) : +- Reused(reference_id=[1]) @@ -701,10 +701,10 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], where=[<(b, 4)], updateAsRetraction=[true], accMode=[Acc]) +- Reused(reference_id=[3]) -Sink(name=[retractSink2], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink2`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[4]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[4]) ]]> @@ -712,7 +712,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac ($0, 6)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) @@ -933,7 +933,7 @@ LogicalSink(name=[retractSink], fields=[a]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) -LogicalSink(name=[upsertSink], fields=[a, b]) +LogicalSink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[<($0, 6)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3]) @@ -948,11 +948,11 @@ Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart= +- Exchange(distribution=[hash[b]], updateAsRetraction=[false], accMode=[Acc]) +- TableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]) -Sink(name=[retractSink], fields=[a], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[a], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[a], where=[>(a, 6)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink], fields=[a, b], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[a, b], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[a, b], where=[<(a, 6)], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -961,7 +961,7 @@ Sink(name=[upsertSink], fields=[a, b], updateAsRetraction=[false], accMode=[AccR diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index b82a3d542874f..29f795f65d7b7 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -218,7 +218,7 @@ Calc(select=[a, b]) =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -63,13 +63,13 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) -Sink(name=[upsertSink1], fields=[b, cnt], updateAsRetraction=[false], accMode=[Acc]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink1`], fields=[b, cnt], updateAsRetraction=[false], accMode=[Acc]) +- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(b) AS frequency], updateAsRetraction=[false], accMode=[Acc]) +- Exchange(distribution=[hash[cnt]], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink2], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink2`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -78,14 +78,14 @@ Sink(name=[upsertSink2], fields=[b, cnt], updateAsRetraction=[false], accMode=[A @@ -143,14 +143,14 @@ Sink(name=[upsertSink], fields=[total_min], updateAsRetraction=[false], accMode= =($0, 4), <($0, 6))]) +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) @@ -165,11 +165,11 @@ GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[tru +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]) -Sink(name=[retractSink], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`retractSink`], fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract]) +- Reused(reference_id=[1]) -Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +Sink(name=[`default_catalog`.`default_database`.`upsertSink`], fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract]) +- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract]) +- Reused(reference_id=[1]) ]]> @@ -178,7 +178,7 @@ Sink(name=[upsertSink], fields=[b, cnt], updateAsRetraction=[false], accMode=[Ac o.f1 match { diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala index 8861ed0f33fc6..fbb37fa685487 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala @@ -216,7 +216,7 @@ class CalcITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sqlQuery) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("0,0,0", "1,1,1", "2,2,2") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala index 91f792208f355..e60a61cfb178a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala @@ -239,7 +239,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,2,,null", "1,3,,null") @@ -261,7 +261,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("3018-06-10", "2018-06-03", "2018-06-01", "2018-06-02") @@ -283,7 +283,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,3018-06-10", "1,2018-06-03", "1,2018-06-01", "1,2018-06-02") @@ -304,7 +304,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("a") @@ -325,7 +325,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") // output two null @@ -347,7 +347,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,a") @@ -368,7 +368,7 @@ class CorrelateITCase extends StreamingTestBase { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("2,null", "3,null") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala index 824907d84c936..b3c1e9e1ac2d0 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala @@ -38,7 +38,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -55,7 +55,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -72,7 +72,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -93,7 +93,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) @@ -114,7 +114,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq("1", "2", "3", "4", "5", "6") @@ -136,7 +136,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getRawResults.size) @@ -157,7 +157,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq("1", "2", "3", "4", "5", "6") @@ -179,7 +179,7 @@ class Limit0RemoveITCase extends StreamingTestBase() { val result = tEnv.sqlQuery(sql) val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink()) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") assertEquals(0, sink.getAppendResults.size) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala index a422c0658c1ff..c2f0fb552fd0c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala @@ -131,7 +131,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -194,7 +194,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -257,7 +257,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -303,7 +303,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -357,7 +357,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -402,7 +402,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val updatedExpected = List( @@ -537,7 +537,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -596,7 +596,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -656,7 +656,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -718,7 +718,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -789,7 +789,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -853,7 +853,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -933,7 +933,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1006,7 +1006,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1073,7 +1073,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema.getFieldNames, schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(table, "MySink") + tEnv.insertInto("MySink", table) tEnv.execute("test") val expected = List( @@ -1202,7 +1202,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema1.getFieldNames, schema1 .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink1", sink1) - tEnv.insertInto(table1, "MySink1") + tEnv.insertInto("MySink1", table1) tEnv.execute("test") val table2 = tEnv.sqlQuery( @@ -1219,7 +1219,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema2.getFieldNames, schema2 .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink2", sink2) - tEnv.insertInto(table2, "MySink2") + tEnv.insertInto("MySink2", table2) tEnv.execute("test") val expected1 = List( @@ -1276,7 +1276,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema1.getFieldNames, schema1.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink1", sink1) - tEnv.insertInto(table1, "MySink1") + tEnv.insertInto("MySink1", table1) tEnv.execute("test") val table2 = tEnv.sqlQuery( @@ -1293,7 +1293,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode configure(schema2.getFieldNames, schema2.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo)) tEnv.registerTableSink("MySink2", sink2) - tEnv.insertInto(table2, "MySink2") + tEnv.insertInto("MySink2", table2) tEnv.execute("test") val expected1 = List( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala index 38a891ae3590f..2cd326a394f85 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala @@ -265,7 +265,7 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo Array("a", "b", "v"), Array(Types.INT, Types.LONG, Types.STRING)) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala index 077fdfdc0918b..d195f0748166a 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala @@ -202,7 +202,7 @@ class WindowAggregateITCase(mode: StateBackendMode) val sink = new TestingUpsertTableSink(Array(0, 1)).configure(fieldNames, fieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(result, "MySink") + tEnv.insertInto("MySink", result) tEnv.execute("test") val expected = Seq( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala index 758f6537d2975..78e3ce0c85ab6 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala @@ -308,7 +308,7 @@ class AggregateITCase(mode: StateBackendMode) extends StreamingWithStateTestBase Array[String]("c", "bMax"), Array[TypeInformation[_]](Types.STRING, Types.LONG)) tEnv.registerTableSink("testSink", tableSink) - tEnv.insertInto(t, "testSink") + tEnv.insertInto("testSink", t) tEnv.execute("test") val expected = List("A,1", "B,2", "C,3") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala index 306b5597c7c33..f1bd82728051b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala @@ -135,7 +135,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode Array[TypeInformation[_]](Types.INT, Types.LONG, Types.LONG)) tEnv.registerTableSink("upsertSink", sink) - tEnv.insertInto(t, "upsertSink") + tEnv.insertInto("upsertSink", t) tEnv.execute("test") val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1") @@ -186,7 +186,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4)) .select('a, 'b, 'c, 'd) tEnv.registerTableSink("retractSink", sink) - tEnv.insertInto(t, "retractSink") + tEnv.insertInto("retractSink", t) tEnv.execute("test") val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3", @@ -793,7 +793,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 2)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("1,5,1,2") @@ -874,7 +874,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingRetractTableSink().configure( schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("1,4,1,2", "1,5,1,2") @@ -1062,7 +1062,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 1)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("MySink", sink) - tEnv.insertInto(t, "MySink") + tEnv.insertInto("MySink", t) tEnv.execute("test") val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1") @@ -1345,7 +1345,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode val sink = new TestingUpsertTableSink(Array(0, 1, 2)) .configure(schema.getFieldNames, schema.getFieldTypes) tEnv.registerTableSink("sinkTests", sink) - tEnv.insertInto(t, "sinkTests") + tEnv.insertInto("sinkTests", t) tEnv.execute("test") val expected = Seq("4,1,1,1") diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala index 7305f5396ab23..ea678b9a4f0de 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala @@ -33,6 +33,7 @@ import org.apache.flink.table.planner.sinks.CollectTableSink import org.apache.flink.table.planner.utils.TableTestUtil import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo import org.apache.flink.util.AbstractID + import _root_.java.util.{UUID, ArrayList => JArrayList} import _root_.scala.collection.JavaConversions._ @@ -45,9 +46,7 @@ object BatchTableEnvUtil { tEnv: TableEnvironment, table: Table, sink: CollectTableSink[T], - jobName: Option[String], - builtInCatalogName: String, - builtInDBName: String): Seq[T] = { + jobName: Option[String]): Seq[T] = { val typeSerializer = fromDataTypeToLegacyInfo(sink.getConsumedDataType) .asInstanceOf[TypeInformation[T]] .createSerializer(tEnv.asInstanceOf[TableEnvironmentImpl] @@ -56,7 +55,7 @@ object BatchTableEnvUtil { sink.init(typeSerializer.asInstanceOf[TypeSerializer[T]], id) val sinkName = UUID.randomUUID().toString tEnv.registerTableSink(sinkName, sink) - tEnv.insertInto(table, builtInCatalogName, builtInDBName, sinkName) + tEnv.insertInto(s"`$sinkName`", table) val res = tEnv.execute("test") val accResult: JArrayList[Array[Byte]] = res.getAccumulatorResult(id) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala index 6b98d2cf8b0bd..1202e224e5af3 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableUtil.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.runtime.utils import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.api.EnvironmentSettings import org.apache.flink.table.api.internal.TableImpl import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType @@ -53,9 +52,7 @@ object TableUtil { new CollectTableSink(_ => t.asInstanceOf[TypeInformation[T]]), Option(jobName)) def collectSink[T]( - table: TableImpl, sink: CollectTableSink[T], jobName: Option[String] = None, - builtInCatalogName: String = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG, - builtInDBName: String = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE): Seq[T] = { + table: TableImpl, sink: CollectTableSink[T], jobName: Option[String] = None): Seq[T] = { // get schema information of table val relNode = TableTestUtil.toRelNode(table) val rowType = relNode.getRowType @@ -74,8 +71,7 @@ object TableUtil { val configuredSink = sink.configure( fieldNames, fieldTypes.map(TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo)) BatchTableEnvUtil.collect(table.getTableEnvironment, - table, configuredSink.asInstanceOf[CollectTableSink[T]], jobName, - builtInCatalogName, builtInDBName) + table, configuredSink.asInstanceOf[CollectTableSink[T]], jobName) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 4ded0fc0eb4c7..c55cd32bb725d 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -33,19 +33,19 @@ import org.apache.flink.table.api.java.internal.{StreamTableEnvironmentImpl => J import org.apache.flink.table.api.java.{StreamTableEnvironment => JavaStreamTableEnv} import org.apache.flink.table.api.scala.internal.{StreamTableEnvironmentImpl => ScalaStreamTableEnvImpl} import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv} -import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog} +import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericInMemoryCatalog, UnresolvedIdentifier} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.delegation.{Executor, ExecutorFactory, PlannerFactory} import org.apache.flink.table.expressions.Expression import org.apache.flink.table.factories.ComponentFactoryService -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableAggregateFunction, TableFunction, UserDefinedAggregateFunction, UserFunctionsTypeHelper} +import org.apache.flink.table.functions._ import org.apache.flink.table.operations.{CatalogSinkModifyOperation, ModifyOperation, QueryOperation} import org.apache.flink.table.planner.calcite.CalciteConfig import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.operations.{DataStreamQueryOperation, PlannerQueryOperation, RichTableSourceQueryOperation} import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.ExecNode -import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkChainedProgram, FlinkStreamProgram, StreamOptimizeContext} +import org.apache.flink.table.planner.plan.optimize.program._ import org.apache.flink.table.planner.plan.stats.FlinkStatistic import org.apache.flink.table.planner.plan.utils.{ExecNodePlanDumper, FlinkRelOptUtil} import org.apache.flink.table.planner.runtime.utils.{TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink} @@ -137,7 +137,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) def writeToSink(table: Table, sink: TableSink[_], sinkName: String): Unit = { val tableEnv = getTableEnv tableEnv.registerTableSink(sinkName, sink) - tableEnv.insertInto(table, sinkName) + tableEnv.insertInto(sinkName, table) } /** @@ -954,10 +954,11 @@ class TestingTableEnvironment private( ) } - override def insertInto(table: Table, path: String, pathContinued: String*): Unit = { - val fullPath = List(path) ++ pathContinued.toList + override def insertInto(path: String, table: Table): Unit = { + val unresolvedIdentifier = parser.parseIdentifier(path) + val identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) - val modifyOperations = List(new CatalogSinkModifyOperation(fullPath, table.getQueryOperation)) + val modifyOperations = List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation)) if (isEagerOperationTranslation) { translate(modifyOperations) } else { @@ -966,7 +967,7 @@ class TestingTableEnvironment private( } override def sqlUpdate(stmt: String): Unit = { - val operations = planner.parse(stmt) + val operations = parser.parse(stmt) if (operations.size != 1) { throw new TableException( "Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type INSERT.") diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java new file mode 100644 index 0000000000000..d862267003ad5 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/calcite/CalciteParser.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.calcite; + +import org.apache.flink.table.api.SqlParserException; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; + +/** + * Thin wrapper around {@link SqlParser} that does exception conversion and {@link SqlNode} casting. + */ +public class CalciteParser { + private final SqlParser.Config config; + + public CalciteParser(SqlParser.Config config) { + this.config = config; + } + + /** + * Parses a SQL statement into a {@link SqlNode}. The {@link SqlNode} is not yet validated. + * + * @param sql a sql string to parse + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the statement + */ + public SqlNode parse(String sql) { + try { + SqlParser parser = SqlParser.create(sql, config); + return parser.parseStmt(); + } catch (SqlParseException e) { + throw new SqlParserException("SQL parse failed. " + e.getMessage()); + } + } + + /** + * Parses a SQL string as an identifier into a {@link SqlIdentifier}. + * + * @param identifier a sql string to parse as an identifier + * @return a parsed sql node + * @throws SqlParserException if an exception is thrown when parsing the identifier + */ + public SqlIdentifier parseIdentifier(String identifier) { + try { + SqlParser parser = SqlParser.create(identifier, config); + SqlNode sqlNode = parser.parseExpression(); + return (SqlIdentifier) sqlNode; + } catch (Exception e) { + throw new SqlParserException(String.format( + "Invalid SQL identifier %s. All SQL keywords must be escaped.", identifier)); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java index 9b50b9191a11b..5be5a7968344e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogCalciteSchema.java @@ -43,12 +43,12 @@ public class CatalogCalciteSchema implements Schema { private final boolean isStreamingMode; private final String catalogName; - private final Catalog catalog; + private final CatalogManager catalogManager; - public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, Catalog catalog) { + public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, CatalogManager catalogManager) { this.isStreamingMode = isStreamingMode; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalogManager; } /** @@ -59,9 +59,8 @@ public CatalogCalciteSchema(boolean isStreamingMode, String catalogName, Catalog */ @Override public Schema getSubSchema(String schemaName) { - - if (catalog.databaseExists(schemaName)) { - return new DatabaseCalciteSchema(isStreamingMode, schemaName, catalogName, catalog); + if (catalogManager.schemaExists(catalogName, schemaName)) { + return new DatabaseCalciteSchema(isStreamingMode, schemaName, catalogName, catalogManager); } else { return null; } @@ -69,7 +68,7 @@ public Schema getSubSchema(String schemaName) { @Override public Set getSubSchemaNames() { - return new HashSet<>(catalog.listDatabases()); + return catalogManager.listSchemas(catalogName); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java index ef1aafc4aa7a3..e7376fc71f45d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/CatalogManagerCalciteSchema.java @@ -82,14 +82,16 @@ public Set getFunctionNames() { @Override public Schema getSubSchema(String name) { - return catalogManager.getCatalog(name) - .map(catalog -> new CatalogCalciteSchema(isStreamingMode, name, catalog)) - .orElse(null); + if (catalogManager.schemaExists(name)) { + return new CatalogCalciteSchema(isStreamingMode, name, catalogManager); + } else { + return null; + } } @Override public Set getSubSchemaNames() { - return catalogManager.getCatalogs(); + return catalogManager.listSchemas(); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java index 76df4bb3b442a..72e89b1ec8dda 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/catalog/DatabaseCalciteSchema.java @@ -19,9 +19,6 @@ package org.apache.flink.table.catalog; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.factories.TableFactory; import org.apache.flink.table.factories.TableFactoryUtil; import org.apache.flink.table.factories.TableSourceFactory; @@ -40,63 +37,62 @@ import org.apache.calcite.schema.Schemas; import org.apache.calcite.schema.Table; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.HashSet; import java.util.Optional; import java.util.Set; -import static java.lang.String.format; - /** * A mapping between Flink catalog's database and Calcite's schema. * Tables are registered as tables in the schema. */ class DatabaseCalciteSchema implements Schema { private final boolean isStreamingMode; - private final String databaseName; private final String catalogName; - private final Catalog catalog; + private final String databaseName; + private final CatalogManager catalogManager; public DatabaseCalciteSchema( boolean isStreamingMode, String databaseName, String catalogName, - Catalog catalog) { + CatalogManager catalogManager) { this.isStreamingMode = isStreamingMode; this.databaseName = databaseName; this.catalogName = catalogName; - this.catalog = catalog; + this.catalogManager = catalogManager; } @Override public Table getTable(String tableName) { + ObjectIdentifier identifier = ObjectIdentifier.of(catalogName, databaseName, tableName); + return catalogManager.getTable(identifier) + .map(result -> { + CatalogBaseTable table = result.getTable(); + final TableFactory tableFactory; + if (result.isTemporary()) { + tableFactory = null; + } else { + tableFactory = catalogManager.getCatalog(catalogName) + .flatMap(Catalog::getTableFactory) + .orElse(null); + } + return convertTable(identifier.toObjectPath(), table, tableFactory); + }) + .orElse(null); + } - ObjectPath tablePath = new ObjectPath(databaseName, tableName); - - try { - if (!catalog.tableExists(tablePath)) { - return null; - } - - CatalogBaseTable table = catalog.getTable(tablePath); - - if (table instanceof QueryOperationCatalogView) { - return QueryOperationCatalogViewTable.createCalciteTable(((QueryOperationCatalogView) table)); - } else if (table instanceof ConnectorCatalogTable) { - return convertConnectorTable((ConnectorCatalogTable) table); - } else if (table instanceof CatalogTable) { - return convertCatalogTable(tablePath, (CatalogTable) table); - } else { - throw new TableException("Unsupported table type: " + table); - } - } catch (TableNotExistException | CatalogException e) { - // TableNotExistException should never happen, because we are checking it exists - // via catalog.tableExists - throw new TableException(format( - "A failure occurred when accessing table. Table path [%s, %s, %s]", - catalogName, - databaseName, - tableName), e); + private Table convertTable(ObjectPath tablePath, CatalogBaseTable table, @Nullable TableFactory tableFactory) { + if (table instanceof QueryOperationCatalogView) { + return QueryOperationCatalogViewTable.createCalciteTable(((QueryOperationCatalogView) table)); + } else if (table instanceof ConnectorCatalogTable) { + return convertConnectorTable((ConnectorCatalogTable) table); + } else if (table instanceof CatalogTable) { + return convertCatalogTable(tablePath, (CatalogTable) table, tableFactory); + } else { + throw new TableException("Unsupported table type: " + table); } } @@ -122,16 +118,14 @@ private Table convertConnectorTable(ConnectorCatalogTable table) { } } - private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { - TableSource tableSource; - Optional tableFactory = catalog.getTableFactory(); - if (tableFactory.isPresent()) { - TableFactory tf = tableFactory.get(); - if (tf instanceof TableSourceFactory) { - tableSource = ((TableSourceFactory) tf).createTableSource(tablePath, table); + private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table, @Nullable TableFactory tableFactory) { + final TableSource tableSource; + if (tableFactory != null) { + if (tableFactory instanceof TableSourceFactory) { + tableSource = ((TableSourceFactory) tableFactory).createTableSource(tablePath, table); } else { - throw new TableException(String.format("Cannot query a sink-only table. TableFactory provided by catalog %s must implement TableSourceFactory", - catalog.getClass())); + throw new TableException( + "Cannot query a sink-only table. TableFactory provided by catalog must implement TableSourceFactory"); } } else { tableSource = TableFactoryUtil.findAndCreateTableSource(table); @@ -153,11 +147,7 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { @Override public Set getTableNames() { - try { - return new HashSet<>(catalog.listTables(databaseName)); - } catch (DatabaseNotExistException e) { - throw new CatalogException(e); - } + return catalogManager.listTables(catalogName, databaseName); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java index 1d12fb9e6204a..3709dd564095b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/plan/QueryOperationConverter.java @@ -264,7 +264,7 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { @Override public RelNode visit(CatalogQueryOperation catalogTable) { - ObjectIdentifier objectIdentifier = catalogTable.getObjectIdentifier(); + ObjectIdentifier objectIdentifier = catalogTable.getTableIdentifier(); return relBuilder.scan( objectIdentifier.getCatalogName(), objectIdentifier.getDatabaseName(), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java new file mode 100644 index 0000000000000..92f0670da81b1 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/ParserImpl.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.calcite.CalciteParser; +import org.apache.flink.table.calcite.FlinkPlannerImpl; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.sqlexec.SqlToOperationConverter; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; + +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +/** + * Implementation of {@link Parser} that uses Calcite. + */ +public class ParserImpl implements Parser { + + private final CatalogManager catalogManager; + + // we use supplier pattern here in order to use the most up to + // date configuration. Users might change the parser configuration in a TableConfig in between + // multiple statements parsing + private final Supplier validatorSupplier; + private final Supplier calciteParserSupplier; + + public ParserImpl( + CatalogManager catalogManager, + Supplier validatorSupplier, + Supplier calciteParserSupplier) { + this.catalogManager = catalogManager; + this.validatorSupplier = validatorSupplier; + this.calciteParserSupplier = calciteParserSupplier; + } + + @Override + public List parse(String statement) { + CalciteParser parser = calciteParserSupplier.get(); + FlinkPlannerImpl planner = validatorSupplier.get(); + // parse the sql query + SqlNode parsed = parser.parse(statement); + + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, parsed) + .orElseThrow(() -> new TableException( + "Unsupported SQL query! parse() only accepts SQL queries of type " + + "SELECT, UNION, INTERSECT, EXCEPT, VALUES, ORDER_BY or INSERT;" + + "and SQL DDLs of type " + + "CREATE TABLE")); + return Collections.singletonList(operation); + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + CalciteParser parser = calciteParserSupplier.get(); + SqlIdentifier sqlIdentifier = parser.parseIdentifier(identifier); + return UnresolvedIdentifier.of(sqlIdentifier.names.toArray(new String[0])); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java index 1c137d157f806..bccf95dd5c431 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/PlanningConfigurationBuilder.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.calcite.CalciteConfig; +import org.apache.flink.table.calcite.CalciteParser; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.calcite.FlinkRelBuilder; import org.apache.flink.table.calcite.FlinkRelBuilderFactory; @@ -133,6 +134,15 @@ public FlinkPlannerImpl createFlinkPlanner(String currentCatalog, String current typeFactory); } + /** + * Creates a configured instance of {@link CalciteParser}. + * + * @return configured calcite parser + */ + public CalciteParser createCalciteParser() { + return new CalciteParser(getSqlParserConfig()); + } + /** Returns the Calcite {@link org.apache.calcite.plan.RelOptPlanner} that will be used. */ public RelOptPlanner getPlanner() { return planner; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java index db8221254bd98..6c0af99a5e348 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sqlexec/SqlToOperationConverter.java @@ -25,10 +25,14 @@ import org.apache.flink.sql.parser.dml.RichSqlInsert; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.calcite.FlinkTypeFactory; +import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.CatalogSinkModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.PlannerQueryOperation; @@ -46,6 +50,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -61,11 +66,15 @@ */ public class SqlToOperationConverter { private FlinkPlannerImpl flinkPlanner; + private CatalogManager catalogManager; //~ Constructors ----------------------------------------------------------- - private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { + private SqlToOperationConverter( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager) { this.flinkPlanner = flinkPlanner; + this.catalogManager = catalogManager; } /** @@ -76,21 +85,27 @@ private SqlToOperationConverter(FlinkPlannerImpl flinkPlanner) { * @param flinkPlanner FlinkPlannerImpl to convert sql node to rel node * @param sqlNode SqlNode to execute on */ - public static Operation convert(FlinkPlannerImpl flinkPlanner, SqlNode sqlNode) { + public static Optional convert( + FlinkPlannerImpl flinkPlanner, + CatalogManager catalogManager, + SqlNode sqlNode) { // validate the query final SqlNode validated = flinkPlanner.validate(sqlNode); - SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner); + SqlToOperationConverter converter = new SqlToOperationConverter(flinkPlanner, catalogManager); if (validated instanceof SqlCreateTable) { - return converter.convertCreateTable((SqlCreateTable) validated); + return Optional.of(converter.convertCreateTable((SqlCreateTable) validated)); } if (validated instanceof SqlDropTable) { - return converter.convertDropTable((SqlDropTable) validated); + return Optional.of(converter.convertDropTable((SqlDropTable) validated)); } else if (validated instanceof RichSqlInsert) { - return converter.convertSqlInsert((RichSqlInsert) validated); + SqlNodeList targetColumnList = ((RichSqlInsert) validated).getTargetColumnList(); + if (targetColumnList != null && targetColumnList.size() != 0) { + throw new ValidationException("Partial inserts are not supported"); + } + return Optional.of(converter.convertSqlInsert((RichSqlInsert) validated)); } else if (validated.getKind().belongsTo(SqlKind.QUERY)) { - return converter.convertSqlQuery(validated); + return Optional.of(converter.convertSqlQuery(validated)); } else { - throw new TableException("Unsupported node type " - + validated.getClass().getSimpleName()); + return Optional.empty(); } } @@ -129,13 +144,22 @@ private Operation convertCreateTable(SqlCreateTable sqlCreateTable) { partitionKeys, properties, tableComment); - return new CreateTableOperation(sqlCreateTable.fullTableName(), catalogTable, + + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlCreateTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + return new CreateTableOperation( + identifier, + catalogTable, sqlCreateTable.isIfNotExists()); } /** Convert DROP TABLE statement. */ private Operation convertDropTable(SqlDropTable sqlDropTable) { - return new DropTableOperation(sqlDropTable.fullTableName(), sqlDropTable.getIfExists()); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(sqlDropTable.fullTableName()); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + return new DropTableOperation(identifier, sqlDropTable.getIfExists()); } /** Fallback method for sql query. */ @@ -147,10 +171,20 @@ private Operation convertSqlQuery(SqlNode node) { private Operation convertSqlInsert(RichSqlInsert insert) { // get name of sink table List targetTablePath = ((SqlIdentifier) insert.getTargetTable()).names; + + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(targetTablePath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + + PlannerQueryOperation query = (PlannerQueryOperation) SqlToOperationConverter.convert( + flinkPlanner, + catalogManager, + insert.getSource()) + .orElseThrow(() -> new TableException( + "Unsupported node type " + insert.getSource().getClass().getSimpleName())); + return new CatalogSinkModifyOperation( - targetTablePath, - (PlannerQueryOperation) SqlToOperationConverter.convert(flinkPlanner, - insert.getSource()), + identifier, + query, insert.getStaticPartitionKVs(), insert.isOverwrite()); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala index 789948a66f304..815cbb4fe0571 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/BatchTableEnvImpl.scala @@ -18,8 +18,6 @@ package org.apache.flink.table.api.internal -import org.apache.calcite.plan.RelOptUtil -import org.apache.calcite.rel.RelNode import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation @@ -28,8 +26,8 @@ import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.table.api._ import org.apache.flink.table.calcite.{CalciteConfig, FlinkTypeFactory} -import org.apache.flink.table.catalog.CatalogManager -import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectorDescriptor} +import org.apache.flink.table.catalog.{CatalogBaseTable, CatalogManager} +import org.apache.flink.table.descriptors.{BatchTableDescriptor, ConnectTableDescriptor, ConnectorDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor import org.apache.flink.table.expressions.{Expression, UnresolvedCallExpression} @@ -48,6 +46,9 @@ import org.apache.flink.table.typeutils.FieldInfoUtils.{getFieldsInfo, validateI import org.apache.flink.table.utils.TableConnectorUtils import org.apache.flink.types.Row +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.RelNode + import _root_.scala.collection.JavaConverters._ /** @@ -68,6 +69,25 @@ abstract class BatchTableEnvImpl( planningConfigurationBuilder ) + /** + * Provides necessary methods for [[ConnectTableDescriptor]]. + */ + private val registration = new Registration() { + override def createTemporaryTable(path: String, table: CatalogBaseTable): Unit = { + val unresolvedIdentifier = parseIdentifier(path) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + catalogManager.createTemporaryTable(table, objectIdentifier, false) + } + + override def createTableSource(name: String, tableSource: TableSource[_]): Unit = { + registerTableSource(name, tableSource) + } + + override def createTableSink(name: String, tableSource: TableSink[_]): Unit = { + registerTableSink(name, tableSource) + } + } + /** * Registers an internal [[BatchTableSource]] in this [[TableEnvImpl]]'s catalog without * name checking. Registered tables can be referenced in SQL queries. @@ -93,7 +113,7 @@ abstract class BatchTableEnvImpl( } def connect(connectorDescriptor: ConnectorDescriptor): BatchTableDescriptor = { - new BatchTableDescriptor(this, connectorDescriptor) + new BatchTableDescriptor(registration, connectorDescriptor) } /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala index ad14eca3c48d2..ca2ecccd170d9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala @@ -20,32 +20,33 @@ package org.apache.flink.table.api.internal import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.sql.parser.ddl.{SqlCreateTable, SqlDropTable} -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.table.api._ -import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder} +import org.apache.flink.table.calcite.{CalciteParser, FlinkPlannerImpl, FlinkRelBuilder} import org.apache.flink.table.catalog._ +import org.apache.flink.table.delegation.Parser import org.apache.flink.table.expressions._ import org.apache.flink.table.expressions.resolver.lookups.TableReferenceLookup import org.apache.flink.table.factories.{TableFactoryService, TableFactoryUtil, TableSinkFactory} import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction, UserDefinedAggregateFunction, _} import org.apache.flink.table.module.{Module, ModuleManager} -import org.apache.flink.table.operations.ddl.CreateTableOperation +import org.apache.flink.table.operations.ddl.{CreateTableOperation, DropTableOperation} import org.apache.flink.table.operations.utils.OperationTreeBuilder -import org.apache.flink.table.operations.{CatalogQueryOperation, PlannerQueryOperation, TableSourceQueryOperation, _} -import org.apache.flink.table.planner.PlanningConfigurationBuilder +import org.apache.flink.table.operations.{CatalogQueryOperation, TableSourceQueryOperation, _} +import org.apache.flink.table.planner.{ParserImpl, PlanningConfigurationBuilder} import org.apache.flink.table.sinks.{OverwritableTableSink, PartitionableTableSink, TableSink, TableSinkUtils} import org.apache.flink.table.sources.TableSource -import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.util.JavaScalaConversionUtil + import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema -import org.apache.calcite.sql._ import org.apache.calcite.sql.parser.SqlParser import org.apache.calcite.tools.FrameworkConfig + +import _root_.java.util.function.{Supplier => JSupplier} import _root_.java.util.{Optional, HashMap => JHashMap, Map => JMap} -import _root_.scala.collection.JavaConversions._ import _root_.scala.collection.JavaConverters._ +import _root_.scala.collection.JavaConversions._ +import _root_.scala.util.Try /** * The abstract base class for the implementation of batch TableEnvironment. @@ -73,7 +74,18 @@ abstract class TableEnvImpl( new TableReferenceLookup { override def lookupTable(name: String): Optional[TableReferenceExpression] = { JavaScalaConversionUtil - .toJava(scanInternal(Array(name)).map(t => new TableReferenceExpression(name, t))) + .toJava( + // The TableLookup is used during resolution of expressions and it actually might not + // be an identifier of a table. It might be a reference to some other object such as + // column, local reference etc. This method should return empty optional in such cases + // to fallback for other identifiers resolution. + Try({ + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + scanInternal(unresolvedIdentifier) + .map(t => new TableReferenceExpression(name, t)) + }) + .toOption + .flatten) } } } @@ -90,6 +102,19 @@ abstract class TableEnvImpl( asRootSchema(new CatalogManagerCalciteSchema(catalogManager, isStreamingMode)), expressionBridge) + private val parser: Parser = new ParserImpl( + catalogManager, + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = getFlinkPlanner + }, + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + new JSupplier[CalciteParser] { + override def get(): CalciteParser = planningConfigurationBuilder.createCalciteParser() + } + ) + def getConfig: TableConfig = config private def isStreamingMode: Boolean = this match { @@ -182,15 +207,32 @@ abstract class TableEnvImpl( } override def registerTable(name: String, table: Table): Unit = { + createTemporaryView(UnresolvedIdentifier.of(name), table) + } + protected def parseIdentifier(identifier: String): UnresolvedIdentifier = { + val parser = planningConfigurationBuilder.createCalciteParser() + UnresolvedIdentifier.of(parser.parseIdentifier(identifier).names: _*) + } + + override def createTemporaryView(path: String, view: Table): Unit = { + val identifier = parseIdentifier(path) + createTemporaryView(identifier, view) + } + + private def createTemporaryView(identifier: UnresolvedIdentifier, view: Table): Unit = { // check that table belongs to this table environment - if (table.asInstanceOf[TableImpl].getTableEnvironment != this) { + if (view.asInstanceOf[TableImpl].getTableEnvironment != this) { throw new TableException( - "Only tables that belong to this TableEnvironment can be registered.") + "Only table API objects that belong to this TableEnvironment can be registered.") } - val view = new QueryOperationCatalogView(table.getQueryOperation) - catalogManager.createTable(view, getTemporaryObjectIdentifier(name), false) + val objectIdentifier = catalogManager.qualifyIdentifier(identifier) + + catalogManager.createTemporaryTable( + new QueryOperationCatalogView(view.getQueryOperation), + objectIdentifier, + false) } override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = { @@ -254,13 +296,13 @@ abstract class TableEnvImpl( protected def validateTableSink(tableSink: TableSink[_]): Unit private def registerTableSourceInternal( - name: String, - tableSource: TableSource[_]) - : Unit = { - // register - getCatalogTable( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, name) match { + name: String, + tableSource: TableSource[_]) + : Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + // check if a table (source or sink) is registered + getTemporaryTable(objectIdentifier) match { // check if a table (source or sink) is registered case Some(table: ConnectorCatalogTable[_, _]) => @@ -274,25 +316,27 @@ abstract class TableEnvImpl( tableSource, table.getTableSink.get, isBatchTable) - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable( + sourceAndSink, + objectIdentifier, + true) } // no table is registered case _ => val source = ConnectorCatalogTable.source(tableSource, isBatchTable) - catalogManager.createTable(source, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(source, objectIdentifier, false) } } private def registerTableSinkInternal( - name: String, - tableSink: TableSink[_]) - : Unit = { + name: String, + tableSink: TableSink[_]) + : Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(name) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) // check if a table (source or sink) is registered - getCatalogTable( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, - name) match { + getTemporaryTable(objectIdentifier) match { // table source and/or sink is registered case Some(table: ConnectorCatalogTable[_, _]) => @@ -306,36 +350,43 @@ abstract class TableEnvImpl( table.getTableSource.get, tableSink, isBatchTable) - catalogManager.alterTable(sourceAndSink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable( + sourceAndSink, + objectIdentifier, + true) } // no table is registered case _ => val sink = ConnectorCatalogTable.sink(tableSink, isBatchTable) - catalogManager.createTable(sink, getTemporaryObjectIdentifier(name), false) + catalogManager.createTemporaryTable(sink, objectIdentifier, false) } } - private def getTemporaryObjectIdentifier(name: String): ObjectIdentifier = { - catalogManager.qualifyIdentifier( - catalogManager.getBuiltInCatalogName, - catalogManager.getBuiltInDatabaseName, - name - ) - } - @throws[TableException] override def scan(tablePath: String*): Table = { - scanInternal(tablePath.toArray) match { + val unresolvedIdentifier = UnresolvedIdentifier.of(tablePath: _*) + scanInternal(unresolvedIdentifier) match { + case Some(table) => createTable(table) + case None => throw new TableException(s"Table '$unresolvedIdentifier' was not found.") + } + } + + override def from(path: String): Table = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + scanInternal(unresolvedIdentifier) match { case Some(table) => createTable(table) - case None => throw new TableException(s"Table '${tablePath.mkString(".")}' was not found.") + case None => throw new TableException(s"Table '$unresolvedIdentifier' was not found.") } } - private[flink] def scanInternal(tablePath: Array[String]): Option[CatalogQueryOperation] = { - val objectIdentifier = catalogManager.qualifyIdentifier(tablePath: _*) + private[flink] def scanInternal(identifier: UnresolvedIdentifier) + : Option[CatalogQueryOperation] = { + val objectIdentifier: ObjectIdentifier = catalogManager.qualifyIdentifier(identifier) + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) - .map(t => new CatalogQueryOperation(objectIdentifier, t.getSchema)) + .map(t => new CatalogQueryOperation(objectIdentifier, t.getTable.getSchema)) } override def listModules(): Array[String] = { @@ -343,7 +394,10 @@ abstract class TableEnvImpl( } override def listCatalogs(): Array[String] = { - catalogManager.getCatalogs.asScala.toArray + catalogManager.listCatalogs + .asScala + .toArray + .sorted } override def listDatabases(): Array[String] = { @@ -354,13 +408,33 @@ abstract class TableEnvImpl( } override def listTables(): Array[String] = { - val currentCatalogName = catalogManager.getCurrentCatalog - val currentCatalog = catalogManager.getCatalog(currentCatalogName) - JavaScalaConversionUtil.toScala(currentCatalog) match { - case Some(catalog) => catalog.listTables(catalogManager.getCurrentDatabase).asScala.toArray - case None => - throw new TableException(s"The current catalog ($currentCatalogName) does not exist.") - } + catalogManager.listTables().asScala + .toArray + .sorted + } + + override def listTemporaryTables(): Array[String] = { + catalogManager.listTemporaryTables().asScala + .toArray + .sorted + } + + override def listTemporaryViews(): Array[String] = { + catalogManager.listTemporaryViews().asScala + .toArray + .sorted + } + + override def dropTemporaryTable(path: String): Boolean = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + catalogManager.dropTemporaryTable(unresolvedIdentifier) + } + + override def dropTemporaryView(path: String): Boolean = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + catalogManager.dropTemporaryView(unresolvedIdentifier) } override def listUserDefinedFunctions(): Array[String] = functionCatalog.getUserDefinedFunctions @@ -375,60 +449,45 @@ abstract class TableEnvImpl( } override def sqlQuery(query: String): Table = { - val planner = getFlinkPlanner - // parse the sql query - val parsed = planner.parse(query) - if (null != parsed && parsed.getKind.belongsTo(SqlKind.QUERY)) { - // validate the sql query - val validated = planner.validate(parsed) - // transform to a relational tree - val relational = planner.rel(validated) - createTable(new PlannerQueryOperation(relational.rel)) - } else { - throw new TableException( - "Unsupported SQL query! sqlQuery() only accepts SQL queries of type " + + val operations = parser.parse(query) + + if (operations.size != 1) throw new ValidationException( + "Unsupported SQL query! sqlQuery() only accepts a single SQL query.") + + operations.get(0) match { + case op: QueryOperation if !op.isInstanceOf[ModifyOperation] => + createTable(op) + case _ => throw new ValidationException( + "Unsupported SQL query! sqlQuery() only accepts a single SQL query of type " + "SELECT, UNION, INTERSECT, EXCEPT, VALUES, and ORDER_BY.") } } override def sqlUpdate(stmt: String): Unit = { - val planner = getFlinkPlanner - // parse the sql query - val parsed = planner.parse(stmt) - parsed match { - case insert: RichSqlInsert => - // validate the insert - val validatedInsert = planner.validate(insert).asInstanceOf[RichSqlInsert] - // we do not validate the row type for sql insert now, so validate the source - // separately. - val validatedQuery = planner.validate(validatedInsert.getSource) - - val tableOperation = new PlannerQueryOperation(planner.rel(validatedQuery).rel) - // get query result as Table - val queryResult = createTable(tableOperation) - - // get name of sink table - val targetTablePath = insert.getTargetTable.asInstanceOf[SqlIdentifier].names - - // insert query result into sink table - insertInto(queryResult, InsertOptions(insert.getStaticPartitionKVs, insert.isOverwrite), - targetTablePath.asScala:_*) - case createTable: SqlCreateTable => - val operation = SqlToOperationConverter - .convert(planner, createTable) - .asInstanceOf[CreateTableOperation] - val objectIdentifier = catalogManager.qualifyIdentifier(operation.getTablePath: _*) + val operations = parser.parse(stmt) + + if (operations.size != 1) throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type " + + "INSERT, CREATE TABLE, DROP TABLE") + + operations.get(0) match { + case op: CatalogSinkModifyOperation => + insertInto( + createTable(op.getChild), + InsertOptions(op.getStaticPartitions, op.isOverwrite), + op.getTableIdentifier) + case createTableOperation: CreateTableOperation => catalogManager.createTable( - operation.getCatalogTable, - objectIdentifier, - operation.isIgnoreIfExists) - case dropTable: SqlDropTable => - val objectIdentifier = catalogManager.qualifyIdentifier(dropTable.fullTableName(): _*) - catalogManager.dropTable(objectIdentifier, dropTable.getIfExists) - case _ => - throw new TableException( - "Unsupported SQL query! sqlUpdate() only accepts SQL statements of " + - "type INSERT, CREATE TABLE, DROP TABLE.") + createTableOperation.getCatalogTable, + createTableOperation.getTableIdentifier, + createTableOperation.isIgnoreIfExists) + case dropTableOperation: DropTableOperation => + catalogManager.dropTable( + dropTableOperation.getTableIdentifier, + dropTableOperation.isIfExists) + case _ => throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of " + + "type INSERT, CREATE TABLE, DROP TABLE") } } @@ -449,14 +508,26 @@ abstract class TableEnvImpl( */ private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit + override def insertInto(path: String, table: Table): Unit = { + val parser = planningConfigurationBuilder.createCalciteParser() + val unresolvedIdentifier = UnresolvedIdentifier.of(parser.parseIdentifier(path).names: _*) + val objectIdentifier: ObjectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) + insertInto( + table, + InsertOptions(new JHashMap[String, String](), overwrite = false), + objectIdentifier) + } + override def insertInto( - table: Table, - path: String, - pathContinued: String*): Unit = { + table: Table, + sinkPath: String, + sinkPathContinued: String*): Unit = { + val unresolvedIdentifier = UnresolvedIdentifier.of(sinkPath +: sinkPathContinued: _*) + val objectIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier) insertInto( table, - InsertOptions(new JHashMap[String, String](), false), - path +: pathContinued: _*) + InsertOptions(new JHashMap[String, String](), overwrite = false), + objectIdentifier) } /** Insert options for executing sql insert. **/ @@ -466,25 +537,24 @@ abstract class TableEnvImpl( * Writes the [[Table]] to a [[TableSink]] that was registered under the specified name. * * @param table The table to write to the TableSink. - * @param sinkTablePath The name of the registered TableSink. + * @param sinkIdentifier The name of the registered TableSink. */ - private def insertInto(table: Table, + private def insertInto( + table: Table, insertOptions: InsertOptions, - sinkTablePath: String*): Unit = { + sinkIdentifier: ObjectIdentifier): Unit = { - val objectIdentifier = catalogManager.qualifyIdentifier(sinkTablePath: _*) - - getTableSink(objectIdentifier) match { + getTableSink(sinkIdentifier) match { case None => - throw new TableException(s"No table was registered under the name $sinkTablePath.") + throw new TableException(s"No table was registered under the name $sinkIdentifier.") case Some(tableSink) => // validate schema of source table and table sink TableSinkUtils.validateSink( insertOptions.staticPartitions, table.getQueryOperation, - objectIdentifier, + sinkIdentifier, tableSink) // set static partitions if it is a partitioned table sink tableSink match { @@ -507,7 +577,8 @@ abstract class TableEnvImpl( } private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => JavaScalaConversionUtil @@ -534,9 +605,10 @@ abstract class TableEnvImpl( } } - protected def getCatalogTable(name: String*): Option[CatalogBaseTable] = { - val objectIdentifier = catalogManager.qualifyIdentifier(name: _*) - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + protected def getTemporaryTable(identifier: ObjectIdentifier): Option[CatalogBaseTable] = { + JavaScalaConversionUtil.toScala(catalogManager.getTable(identifier)) + .filter(_.isTemporary) + .map(_.getTable) } /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */ diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala index 971ba3718ac17..f79a3ba9555a3 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/java/internal/BatchTableEnvironmentImpl.scala @@ -69,6 +69,19 @@ class BatchTableEnvironmentImpl( registerTable(name, fromDataSet(dataSet, fields)) } + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T]): Unit = { + createTemporaryView(path, fromDataSet(dataSet)) + } + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T], + fields: String): Unit = { + createTemporaryView(path, fromDataSet(dataSet, fields)) + } + override def toDataSet[T](table: Table, clazz: Class[T]): DataSet[T] = { // Use the default query config. translate[T](table)(TypeExtractor.createTypeInfo(clazz)) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala index 7324a4bbccd3c..1f0a4a0329747 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/scala/internal/BatchTableEnvironmentImpl.scala @@ -92,5 +92,18 @@ class BatchTableEnvironmentImpl( queryConfig: BatchQueryConfig, sinkPath: String, sinkPathContinued: String*): Unit = insertInto(table, sinkPath, sinkPathContinued: _*) + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T]): Unit = { + createTemporaryView(path, fromDataSet(dataSet)) + } + + override def createTemporaryView[T]( + path: String, + dataSet: DataSet[T], + fields: Expression*): Unit = { + createTemporaryView(path, fromDataSet(dataSet, fields: _*)) + } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala index 285631645be10..8ce6569582e83 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala @@ -19,10 +19,9 @@ package org.apache.flink.table.calcite import org.apache.flink.sql.parser.ExtendedSqlNode -import org.apache.flink.table.api.{SqlParserException, TableException, ValidationException} +import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.catalog.CatalogReader -import com.google.common.collect.ImmutableList import org.apache.calcite.plan.RelOptTable.ViewExpander import org.apache.calcite.plan._ import org.apache.calcite.prepare.CalciteCatalogReader @@ -30,9 +29,7 @@ import org.apache.calcite.rel.RelRoot import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.RelFactories import org.apache.calcite.rex.RexBuilder -import org.apache.calcite.schema.SchemaPlus import org.apache.calcite.sql.advise.{SqlAdvisor, SqlAdvisorValidator} -import org.apache.calcite.sql.parser.{SqlParser, SqlParseException => CSqlParseException} import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql.{SqlKind, SqlNode, SqlOperatorTable} import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter} @@ -57,24 +54,13 @@ class FlinkPlannerImpl( val typeFactory: FlinkTypeFactory) { val operatorTable: SqlOperatorTable = config.getOperatorTable - /** Holds the trait definitions to be registered with planner. May be null. */ - val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs - val parserConfig: SqlParser.Config = config.getParserConfig val convertletTable: SqlRexConvertletTable = config.getConvertletTable val sqlToRelConverterConfig: SqlToRelConverter.Config = config.getSqlToRelConverterConfig + val parser = new CalciteParser(config.getParserConfig) var validator: FlinkCalciteSqlValidator = _ var root: RelRoot = _ - private def ready() { - if (this.traitDefs != null) { - planner.clearRelTraitDefs() - for (traitDef <- this.traitDefs) { - planner.addRelTraitDef(traitDef) - } - } - } - def getCompletionHints(sql: String, cursor: Int): Array[String] = { val advisorValidator = new SqlAdvisorValidator( operatorTable, @@ -111,18 +97,6 @@ class FlinkPlannerImpl( validator } - def parse(sql: String): SqlNode = { - try { - ready() - val parser: SqlParser = SqlParser.create(sql, parserConfig) - val sqlNode: SqlNode = parser.parseStmt - sqlNode - } catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } - } - def validate(sqlNode: SqlNode): SqlNode = { val catalogReader = catalogReaderSupplier.apply(false) // do pre-validate rewrite. @@ -185,15 +159,7 @@ class FlinkPlannerImpl( schemaPath: util.List[String], viewPath: util.List[String]): RelRoot = { - val parser: SqlParser = SqlParser.create(queryString, parserConfig) - var sqlNode: SqlNode = null - try { - sqlNode = parser.parseQuery - } - catch { - case e: CSqlParseException => - throw new SqlParserException(s"SQL parse failed. ${e.getMessage}", e) - } + val sqlNode = parser.parse(queryString) val catalogReader: CalciteCatalogReader = catalogReaderSupplier.apply(false) .withSchemaPath(schemaPath) val validator: SqlValidator = @@ -227,14 +193,3 @@ class FlinkPlannerImpl( RelFactories.LOGICAL_BUILDER.create(relOptCluster, relOptSchema) } } - -object FlinkPlannerImpl { - private def rootSchema(schema: SchemaPlus): SchemaPlus = { - if (schema.getParentSchema == null) { - schema - } - else { - rootSchema(schema.getParentSchema) - } - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala index c50d32b61595a..4c54ec600b41a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/StreamPlanner.scala @@ -21,13 +21,12 @@ import org.apache.flink.annotation.VisibleForTesting import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} -import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.streaming.api.datastream.{DataStream, DataStreamSink} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api._ -import org.apache.flink.table.calcite.{CalciteConfig, FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory} +import org.apache.flink.table.calcite._ import org.apache.flink.table.catalog.{CatalogManager, CatalogManagerCalciteSchema, CatalogTable, ConnectorCatalogTable, _} -import org.apache.flink.table.delegation.{Executor, Planner} +import org.apache.flink.table.delegation.{Executor, Parser, Planner} import org.apache.flink.table.executor.StreamExecutor import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.{ExpressionBridge, PlannerExpression, PlannerExpressionConverter, PlannerTypeInferenceUtilImpl} @@ -39,7 +38,6 @@ import org.apache.flink.table.plan.nodes.datastream.DataStreamRel import org.apache.flink.table.plan.util.UpdatingPlanChecker import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.sinks._ -import org.apache.flink.table.sqlexec.SqlToOperationConverter import org.apache.flink.table.types.utils.TypeConversions import org.apache.flink.table.util.JavaScalaConversionUtil @@ -47,11 +45,11 @@ import org.apache.calcite.jdbc.CalciteSchema import org.apache.calcite.jdbc.CalciteSchemaBuilder.asRootSchema import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rel.RelNode -import org.apache.calcite.sql.SqlKind import _root_.java.lang.{Boolean => JBool} import _root_.java.util -import _root_.java.util.{Objects, List => JList} +import _root_.java.util.Objects +import _root_.java.util.function.{Supplier => JSupplier} import _root_.scala.collection.JavaConversions._ import _root_.scala.collection.JavaConverters._ @@ -99,28 +97,20 @@ class StreamPlanner( .orElse(CalciteConfig.DEFAULT), planningConfigurationBuilder) - override def parse(stmt: String): JList[Operation] = { - val planner = getFlinkPlanner - // parse the sql query - val parsed = planner.parse(stmt) - - parsed match { - case insert: RichSqlInsert => - val targetColumnList = insert.getTargetColumnList - if (targetColumnList != null && insert.getTargetColumnList.size() != 0) { - throw new ValidationException("Partial inserts are not supported") - } - List(SqlToOperationConverter.convert(planner, insert)) - case node if node.getKind.belongsTo(SqlKind.QUERY) || node.getKind.belongsTo(SqlKind.DDL) => - List(SqlToOperationConverter.convert(planner, parsed)).asJava - case _ => - throw new TableException( - "Unsupported SQL query! parse() only accepts SQL queries of type " + - "SELECT, UNION, INTERSECT, EXCEPT, VALUES, ORDER_BY or INSERT;" + - "and SQL DDLs of type " + - "CREATE TABLE") + private val parser: Parser = new ParserImpl( + catalogManager, + // we do not cache the parser in order to use the most up to + // date configuration. Users might change parser configuration in TableConfig in between + // parsing statements + new JSupplier[FlinkPlannerImpl] { + override def get(): FlinkPlannerImpl = getFlinkPlanner + }, + new JSupplier[CalciteParser] { + override def get(): CalciteParser = planningConfigurationBuilder.createCalciteParser() } - } + ) + + override def getParser: Parser = parser override def translate(tableOperations: util.List[ModifyOperation]) : util.List[Transformation[_]] = { @@ -151,13 +141,12 @@ class StreamPlanner( writeToSink(s.getChild, s.getSink, unwrapQueryConfig) case catalogSink: CatalogSinkModifyOperation => - val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) - getTableSink(identifier) + getTableSink(catalogSink.getTableIdentifier) .map(sink => { TableSinkUtils.validateSink( catalogSink.getStaticPartitions, catalogSink.getChild, - identifier, + catalogSink.getTableIdentifier, sink) // set static partitions if it is a partitioned sink sink match { @@ -177,7 +166,8 @@ class StreamPlanner( writeToSink(catalogSink.getChild, sink, unwrapQueryConfig) }) match { case Some(t) => t - case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") + case None => + throw new TableException(s"Sink ${catalogSink.getTableIdentifier} does not exists") } case outputConversion: OutputConversionModifyOperation => @@ -431,7 +421,8 @@ class StreamPlanner( } private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { - JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { + JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) + .map(_.getTable) match { case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => JavaScalaConversionUtil.toScala(s.asInstanceOf[ConnectorCatalogTable[_, _]].getTableSink) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java index 14f818a796874..42ccef1f22632 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogManagerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.catalog; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.util.TestLogger; @@ -25,10 +26,19 @@ import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.table.catalog.CatalogStructureBuilder.BUILTIN_CATALOG_NAME; import static org.apache.flink.table.catalog.CatalogStructureBuilder.database; import static org.apache.flink.table.catalog.CatalogStructureBuilder.root; +import static org.apache.flink.table.catalog.CatalogStructureBuilder.table; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -50,13 +60,13 @@ public void testRegisterCatalog() throws Exception { database(BUILTIN_DEFAULT_DATABASE_NAME)) .build(); - assertEquals(1, manager.getCatalogs().size()); - assertFalse(manager.getCatalogs().contains(TEST_CATALOG_NAME)); + assertEquals(1, manager.listCatalogs().size()); + assertFalse(manager.listCatalogs().contains(TEST_CATALOG_NAME)); manager.registerCatalog(TEST_CATALOG_NAME, new GenericInMemoryCatalog(TEST_CATALOG_NAME)); - assertEquals(2, manager.getCatalogs().size()); - assertTrue(manager.getCatalogs().contains(TEST_CATALOG_NAME)); + assertEquals(2, manager.listCatalogs().size()); + assertTrue(manager.listCatalogs().contains(TEST_CATALOG_NAME)); } @Test @@ -91,6 +101,175 @@ public void testRegisterCatalogWithExistingName() throws Exception { manager.registerCatalog(TEST_CATALOG_NAME, new GenericInMemoryCatalog(TEST_CATALOG_NAME)); } + @Test + public void testReplaceTemporaryTable() throws Exception { + ObjectIdentifier tempIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "temp"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME)) + .temporaryTable(tempIdentifier) + .build(); + + CatalogTest.TestTable table = new CatalogTest.TestTable(); + manager.createTemporaryTable(table, tempIdentifier, true); + assertThat(manager.getTable(tempIdentifier).get().isTemporary(), equalTo(true)); + assertThat(manager.getTable(tempIdentifier).get().getTable(), equalTo(table)); + } + + @Test + public void testTemporaryTableExists() throws Exception { + ObjectIdentifier tempIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "temp"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME)) + .temporaryTable(tempIdentifier) + .build(); + + thrown.expect(ValidationException.class); + thrown.expectMessage(String.format("Temporary table %s already exists", tempIdentifier)); + manager.createTemporaryTable(new CatalogTest.TestTable(), tempIdentifier, false); + } + + @Test + public void testDropTableWhenTemporaryTableExists() throws Exception { + ObjectIdentifier identifier = ObjectIdentifier.of(BUILTIN_CATALOG_NAME, BUILTIN_DEFAULT_DATABASE_NAME, "test"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .temporaryTable(identifier) + .build(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Temporary table with identifier '`builtin`.`default`.`test`' exists." + + " Drop it first before removing the permanent table."); + manager.dropTable(identifier, false); + + } + + @Test + public void testDropTemporaryNonExistingTable() throws Exception { + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .build(); + + boolean dropped = manager.dropTemporaryTable(UnresolvedIdentifier.of("test")); + + assertThat(dropped, is(false)); + } + + @Test + public void testDropTemporaryTable() throws Exception { + ObjectIdentifier identifier = ObjectIdentifier.of(BUILTIN_CATALOG_NAME, BUILTIN_DEFAULT_DATABASE_NAME, "test"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test"))) + .temporaryTable(identifier) + .build(); + + boolean dropped = manager.dropTemporaryTable(UnresolvedIdentifier.of("test")); + + assertThat(dropped, is(true)); + } + + @Test + public void testListTables() throws Exception { + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + ObjectIdentifier viewIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "testView"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(identifier1) + .temporaryTable(identifier2) + .temporaryView(viewIdentifier, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(BUILTIN_CATALOG_NAME); + manager.setCurrentDatabase(BUILTIN_DEFAULT_DATABASE_NAME); + + assertThat( + manager.listTables(), + equalTo( + setOf( + "test2", + "testView", + "test_in_builtin" + ))); + } + + @Test + public void testListTemporaryTables() throws Exception { + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + ObjectIdentifier viewIdentifier = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "testView"); + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(identifier1) + .temporaryTable(identifier2) + .temporaryView(viewIdentifier, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(BUILTIN_CATALOG_NAME); + manager.setCurrentDatabase(BUILTIN_DEFAULT_DATABASE_NAME); + + assertThat( + manager.listTemporaryTables(), + equalTo( + setOf( + "test2", + "testView" + ))); + } + + @Test + public void testListTemporaryViews() throws Exception { + ObjectIdentifier tableIdentifier = ObjectIdentifier.of( + TEST_CATALOG_NAME, + TEST_CATALOG_DEFAULT_DB_NAME, + "table"); + ObjectIdentifier identifier1 = ObjectIdentifier.of(TEST_CATALOG_NAME, TEST_CATALOG_DEFAULT_DB_NAME, "test1"); + ObjectIdentifier identifier2 = ObjectIdentifier.of( + BUILTIN_CATALOG_NAME, + BUILTIN_DEFAULT_DATABASE_NAME, + "test2"); + + CatalogManager manager = root() + .builtin( + database(BUILTIN_DEFAULT_DATABASE_NAME, table("test_in_builtin"))) + .catalog(TEST_CATALOG_NAME, database(TEST_CATALOG_DEFAULT_DB_NAME, table("test_in_catalog"))) + .temporaryTable(tableIdentifier) + .temporaryView(identifier1, "SELECT * FROM none") + .temporaryView(identifier2, "SELECT * FROM none") + .build(); + + manager.setCurrentCatalog(TEST_CATALOG_NAME); + manager.setCurrentDatabase(TEST_CATALOG_DEFAULT_DB_NAME); + + assertThat(manager.listTemporaryViews(), equalTo(setOf("test1"))); + } + @Test public void testSetNonExistingCurrentCatalog() throws Exception { thrown.expect(CatalogException.class); @@ -109,4 +288,8 @@ public void testSetNonExistingCurrentDatabase() throws Exception { // This catalog does not exist in the builtin catalog manager.setCurrentDatabase("nonexistent"); } + + private Set setOf(String... element) { + return Stream.of(element).collect(Collectors.toSet()); + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java index 384251f0408fc..5141802888de6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/CatalogStructureBuilder.java @@ -26,8 +26,11 @@ import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.Collections; import java.util.HashMap; +import java.util.Map; import java.util.Objects; +import java.util.Optional; /** * Utility classes to construct a {@link CatalogManager} with a given structure. @@ -51,6 +54,7 @@ * table("tab1"), * table("tab2") * ) + * .temporaryTable(ObjectIdentifier.of("cat1", "default", "tab1")) * ).build(); * } */ @@ -80,11 +84,30 @@ public CatalogStructureBuilder builtin(DatabaseBuilder defaultDb, DatabaseBuilde return this; } + public CatalogStructureBuilder temporaryTable(ObjectIdentifier path) { + this.catalogManager.createTemporaryTable(new TestTable(path.toString(), true), path, false); + return this; + } + + public CatalogStructureBuilder temporaryView(ObjectIdentifier path, String query) { + this.catalogManager.createTemporaryTable( + new TestView( + query, + query, + TableSchema.builder().build(), + Collections.emptyMap(), + "", + true, + path.toString()), + path, + false); + return this; + } + public CatalogStructureBuilder catalog( String name, DatabaseBuilder defaultDatabase, DatabaseBuilder... databases) throws Exception { - GenericInMemoryCatalog catalog = buildCatalog(name, defaultDatabase, databases); catalogManager.registerCatalog(name, catalog); @@ -158,33 +181,46 @@ public String getName() { } public TestTable build(String path) { - return new TestTable(path + "." + name); + return new TestTable(path + "." + name, false); } } - private static class TestTable extends ConnectorCatalogTable { + /** + * A test {@link CatalogTable}. + */ + public static class TestTable extends ConnectorCatalogTable { private final String fullyQualifiedPath; + private final boolean isTemporary; - private static final StreamTableSource tableSource = new StreamTableSource() { - @Override - public DataStream getDataStream(StreamExecutionEnvironment execEnv) { - return null; - } + public boolean isTemporary() { + return isTemporary; + } - @Override - public TypeInformation getReturnType() { - return Types.ROW(); - } + private TestTable(String fullyQualifiedPath, boolean isTemporary) { + super(new StreamTableSource() { + @Override + public DataStream getDataStream(StreamExecutionEnvironment execEnv) { + return null; + } - @Override - public TableSchema getTableSchema() { - return TableSchema.builder().build(); - } - }; + @Override + public TypeInformation getReturnType() { + return Types.ROW(); + } + + @Override + public TableSchema getTableSchema() { + return TableSchema.builder().build(); + } + + @Override + public String explainSource() { + return String.format("isTemporary=[%s]", isTemporary); + } + }, null, TableSchema.builder().build(), false); - private TestTable(String fullyQualifiedPath) { - super(tableSource, null, tableSource.getTableSchema(), false); this.fullyQualifiedPath = fullyQualifiedPath; + this.isTemporary = isTemporary; } @Override @@ -196,12 +232,71 @@ public boolean equals(Object o) { return false; } TestTable testTable = (TestTable) o; - return Objects.equals(fullyQualifiedPath, testTable.fullyQualifiedPath); + return Objects.equals(fullyQualifiedPath, testTable.fullyQualifiedPath) && + Objects.equals(isTemporary, testTable.isTemporary); + } + + @Override + public int hashCode() { + return Objects.hash(fullyQualifiedPath, isTemporary); + } + } + + /** + * A test {@link CatalogView}. + */ + public static class TestView extends AbstractCatalogView { + private final boolean isTemporary; + private final String fullyQualifiedPath; + + public boolean isTemporary() { + return isTemporary; + } + + private TestView( + String originalQuery, + String expandedQuery, + TableSchema schema, + Map properties, + String comment, + boolean isTemporary, + String fullyQualifiedPath) { + super(originalQuery, expandedQuery, schema, properties, comment); + this.isTemporary = isTemporary; + this.fullyQualifiedPath = fullyQualifiedPath; + } + + @Override + public CatalogBaseTable copy() { + return this; + } + + @Override + public Optional getDescription() { + return Optional.empty(); + } + + @Override + public Optional getDetailedDescription() { + return Optional.empty(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestView testView = (TestView) o; + return isTemporary == testView.isTemporary && + Objects.equals(fullyQualifiedPath, testView.fullyQualifiedPath); } @Override public int hashCode() { - return Objects.hash(fullyQualifiedPath); + return Objects.hash(isTemporary, fullyQualifiedPath); } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java index ea447bb16592e..ba06d1c93bf11 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/DatabaseCalciteSchemaTest.java @@ -48,10 +48,11 @@ public class DatabaseCalciteSchemaTest { @Test public void testCatalogTable() throws TableAlreadyExistException, DatabaseNotExistException { GenericInMemoryCatalog catalog = new GenericInMemoryCatalog(catalogName, databaseName); + CatalogManager catalogManager = new CatalogManager(catalogName, catalog); DatabaseCalciteSchema calciteSchema = new DatabaseCalciteSchema(true, databaseName, catalogName, - catalog); + catalogManager); catalog.createTable(new ObjectPath(databaseName, tableName), new TestCatalogBaseTable(), false); Table table = calciteSchema.getTable(tableName); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java index 703f99ea2e673..c2e323f073335 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/catalog/PathResolutionTest.java @@ -43,6 +43,7 @@ import static org.apache.flink.table.catalog.CatalogStructureBuilder.root; import static org.apache.flink.table.catalog.CatalogStructureBuilder.table; import static org.apache.flink.table.catalog.PathResolutionTest.TestSpec.testSpec; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; /** @@ -110,7 +111,13 @@ public static List testData() throws Exception { .withCatalogManager(catalogWithSpecialCharacters()) .tableApiLookupPath("default db", "tab 1") .sqlLookupPath("`default db`.`tab 1`") - .expectPath(BUILTIN_CATALOG_NAME, "default db", "tab 1") + .expectPath(BUILTIN_CATALOG_NAME, "default db", "tab 1"), + + testSpec("shadowingWithTemporaryTable") + .withCatalogManager(catalogWithTemporaryObjects()) + .tableApiLookupPath("cat1", "db1", "tab1") + .sqlLookupPath("cat1.db1.tab1") + .expectTemporaryPath("cat1", "db1", "tab1") ); } @@ -139,6 +146,22 @@ private static CatalogManager simpleCatalog() throws Exception { ).build(); } + private static CatalogManager catalogWithTemporaryObjects() throws Exception { + return root() + .builtin( + database("default") + ) + .catalog( + "cat1", + database( + "db1", + table("tab1") + ) + ) + .temporaryTable(ObjectIdentifier.of("cat1", "db1", "tab1")) + .build(); + } + private static CatalogManager catalogWithSpecialCharacters() throws Exception { return root() .builtin( @@ -168,10 +191,15 @@ public void testTableApiPathResolution() { testSpec.getDefaultCatalog().ifPresent(catalogManager::setCurrentCatalog); testSpec.getDefaultDatabase().ifPresent(catalogManager::setCurrentDatabase); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(lookupPath.toArray(new String[0])); + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(lookupPath.toArray(new String[0])); + ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + assertThat( Arrays.asList(identifier.getCatalogName(), identifier.getDatabaseName(), identifier.getObjectName()), CoreMatchers.equalTo(testSpec.getExpectedPath())); + Optional tableLookup = catalogManager.getTable(identifier); + assertThat(tableLookup.isPresent(), is(true)); + assertThat(tableLookup.get().isTemporary(), is(testSpec.isTemporaryObject())); } @Test @@ -185,8 +213,9 @@ public void testStreamSqlPathResolution() { util.verifyJavaSql( format("SELECT * FROM %s", testSpec.getSqlPathToLookup()), format( - "StreamTableSourceScan(table=[[%s]], fields=[], source=[()])", - String.join(", ", testSpec.getExpectedPath())) + "StreamTableSourceScan(table=[[%s]], fields=[], source=[isTemporary=[%s]])", + String.join(", ", testSpec.getExpectedPath()), + testSpec.isTemporaryObject()) ); } @@ -196,6 +225,7 @@ static class TestSpec { private String sqlPathToLookup; private List tableApiLookupPath; private List expectedPath; + private boolean isTemporaryObject = false; private String defaultCatalog; private String defaultDatabase; private CatalogManager catalogManager; @@ -237,6 +267,11 @@ public TestSpec expectPath(String... expectedPath) { return this; } + public TestSpec expectTemporaryPath(String... expectedPath) { + this.isTemporaryObject = true; + return expectPath(expectedPath); + } + public TestSpec withDefaultPath(String defaultCatalog) { this.defaultCatalog = defaultCatalog; return this; @@ -272,6 +307,10 @@ public Optional getDefaultDatabase() { return Optional.ofNullable(defaultDatabase); } + public boolean isTemporaryObject() { + return isTemporaryObject; + } + @Override public String toString() { @@ -286,6 +325,10 @@ public String toString() { properties.add("defaultDatabase: " + defaultDatabase); } + if (isTemporaryObject) { + properties.add("temporary: true"); + } + properties.add("sqlPath: " + sqlPathToLookup); properties.add("tableApiPath: " + tableApiLookupPath); properties.add("expectedPath: " + expectedPath); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java index 79f5470d557a8..21a399f2781ce 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/sqlexec/SqlToOperationConverterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; +import org.apache.flink.table.calcite.CalciteParser; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogManager; @@ -126,9 +127,9 @@ public void testCreateTable() { " 'kafka.topic' = 'log.test'\n" + ")\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -155,9 +156,9 @@ public void testCreateTableWithMinusInOptionKey() { " 'a.b-c-d.e-f1231.g' = 'ada',\n" + " 'a.b-c-d.*' = 'adad')\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CreateTableOperation; CreateTableOperation op = (CreateTableOperation) operation; CatalogTable catalogTable = op.getCatalogTable(); @@ -189,18 +190,18 @@ public void testCreateTableWithPkUniqueKeys() { " 'kafka.topic' = 'log.test'\n" + ")\n"; final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - SqlToOperationConverter.convert(planner, node); + SqlToOperationConverter.convert(planner, catalogManager, node); } @Test public void testSqlInsertWithStaticPartition() { final String sql = "insert into t1 partition(a=1) select b, c, d from t2"; FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.HIVE); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.HIVE).parse(sql); assert node instanceof RichSqlInsert; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); assert operation instanceof CatalogSinkModifyOperation; CatalogSinkModifyOperation sinkModifyOperation = (CatalogSinkModifyOperation) operation; final Map expectedStaticPartitions = new HashMap<>(); @@ -369,9 +370,9 @@ public void testCreateTableWithFullDataTypes() { } final String sql = buffer.toString(); final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; - Operation operation = SqlToOperationConverter.convert(planner, node); + Operation operation = SqlToOperationConverter.convert(planner, catalogManager, node).get(); TableSchema schema = ((CreateTableOperation) operation).getCatalogTable().getSchema(); Object[] expectedDataTypes = testItems.stream().map(item -> item.expectedType).toArray(); assertArrayEquals(expectedDataTypes, schema.getFieldDataTypes()); @@ -397,11 +398,11 @@ public void testCreateTableWithUnSupportedDataTypes() { final FlinkPlannerImpl planner = getPlannerBySqlDialect(SqlDialect.DEFAULT); for (TestItem item : testItems) { String sql = String.format(sqlTemplate, item.testExpr); - SqlNode node = planner.parse(sql); + SqlNode node = getParserBySqlDialect(SqlDialect.DEFAULT).parse(sql); assert node instanceof SqlCreateTable; expectedEx.expect(TableException.class); expectedEx.expectMessage(item.expectedError); - SqlToOperationConverter.convert(planner, node); + SqlToOperationConverter.convert(planner, catalogManager, node); } } @@ -419,6 +420,11 @@ private static TestItem createTestItem(Object... args) { return testItem; } + private CalciteParser getParserBySqlDialect(SqlDialect sqlDialect) { + tableConfig.setSqlDialect(sqlDialect); + return planningConfigurationBuilder.createCalciteParser(); + } + private FlinkPlannerImpl getPlannerBySqlDialect(SqlDialect sqlDialect) { tableConfig.setSqlDialect(sqlDialect); return planningConfigurationBuilder.createFlinkPlanner(catalogManager.getCurrentCatalog(), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala index 5b7736442e266..b91f0d656026a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableSourceTest.scala @@ -18,17 +18,21 @@ package org.apache.flink.table.api +import _root_.java.util.{HashMap => JHashMap} +import _root_.java.util.{Map => JMap} import _root_.java.sql.{Date, Time, Timestamp} - import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.scala._ +import org.apache.flink.table.descriptors.{ConnectorDescriptor, Schema} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR import org.apache.flink.table.expressions.utils._ import org.apache.flink.table.runtime.utils.CommonTestData import org.apache.flink.table.sources.{CsvTableSource, TableSource} import org.apache.flink.table.utils.TableTestUtil._ import org.apache.flink.table.utils.{TableTestBase, TestFilterableTableSource} import org.apache.flink.types.Row + import org.junit.{Assert, Test} class TableSourceTest extends TableTestBase { @@ -362,6 +366,34 @@ class TableSourceTest extends TableTestBase { util.verifyTable(result, expected) } + @Test + def testConnectToTableWithProperties(): Unit = { + val util = streamTestUtil() + val tableEnv = util.tableEnv + + val path = "cat.db.tab1" + tableEnv.connect(new ConnectorDescriptor("COLLECTION", 1, false) { + override protected def toConnectorProperties: JMap[String, String] = { + val context = new JHashMap[String, String]() + context.put(CONNECTOR, "COLLECTION") + context + } + }).withSchema( + new Schema() + .schema(TableSchema.builder() + .field("id", DataTypes.INT()) + .field("name", DataTypes.STRING()) + .build()) + ).createTemporaryTable(path) + + val result = tableEnv.from(path) + + val expected = "StreamTableSourceScan(table=[[cat, db, tab1]], fields=[id, name], " + + "source=[CollectionTableSource(id, name)])" + + util.verifyTable(result, expected) + } + // csv builder @Test @@ -490,7 +522,6 @@ class TableSourceTest extends TableTestBase { (tableSource, "filterableTable") } - def csvTable: (CsvTableSource, String) = { val csvTable = CommonTestData.getCsvTableSource val tableName = "csvTable" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala index 081fee6be3324..180c83321f4bf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.expressions.utils import java.util import java.util.concurrent.Future - import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, HepProgramBuilder} import org.apache.calcite.rel.RelNode import org.apache.calcite.rex.RexNode @@ -41,13 +40,14 @@ import org.apache.flink.table.api.scala.BatchTableEnvironment import org.apache.flink.table.api.TableConfig import org.apache.flink.table.api.internal.TableEnvImpl import org.apache.flink.table.api.scala.internal.BatchTableEnvironmentImpl -import org.apache.flink.table.calcite.FlinkRelBuilder +import org.apache.flink.table.calcite.{CalciteParser, FlinkRelBuilder} import org.apache.flink.table.codegen.{Compiler, FunctionCodeGenerator, GeneratedFunction} import org.apache.flink.table.expressions.{Expression, ExpressionParser} import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.plan.nodes.dataset.{DataSetCalc, DataSetScan} import org.apache.flink.table.plan.rules.FlinkRuleSets import org.apache.flink.types.Row + import org.junit.Assert._ import org.junit.{After, Before} import org.mockito.Mockito._ @@ -179,7 +179,8 @@ abstract class ExpressionTestBase { private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = { // create RelNode from SQL expression - val parsed = planner.parse(s"SELECT $sqlExpr FROM $tableName") + val parsed = new CalciteParser(context._2.getParserConfig) + .parse(s"SELECT $sqlExpr FROM $tableName") val validated = planner.validate(parsed) val converted = planner.rel(validated).rel diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala index cb7d85a935f1a..6cb549f1b1125 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/match/PatternTranslatorTestBase.scala @@ -31,6 +31,7 @@ import org.apache.flink.table.plan.nodes.datastream.{DataStreamMatch, DataStream import org.apache.flink.table.planner.StreamPlanner import org.apache.flink.types.Row import org.apache.flink.util.TestLogger + import org.junit.Assert._ import org.junit.rules.ExpectedException import org.junit.{ComparisonFailure, Rule} @@ -68,7 +69,7 @@ abstract class PatternTranslatorTestBase extends TestLogger{ def verifyPattern(matchRecognize: String, expected: Pattern[Row, _ <: Row]): Unit = { // create RelNode from SQL expression - val parsed = context._3.parse( + val parsed = context._3.getParser.parse( s""" |SELECT * |FROM $tableName diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index b0f11f3b60797..bec0cebd3f7b3 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -91,6 +91,8 @@ class MockTableEnvironment extends TableEnvironment { override def useDatabase(databaseName: String): Unit = ??? + override def insertInto(sinkPath: String, table: Table): Unit = ??? + override def insertInto( table: Table, sinkPath: String, @@ -101,4 +103,19 @@ class MockTableEnvironment extends TableEnvironment { override def loadModule(moduleName: String, module: Module): Unit = ??? override def unloadModule(moduleName: String): Unit = ??? + + override def createTemporaryView( + path: String, + view: Table): Unit = ??? + + override def listTemporaryTables(): Array[String] = ??? + + override def listTemporaryViews(): Array[String] = ??? + + override def from(path: String): Table = ??? + + override def dropTemporaryTable(path: String): Boolean = ??? + + override def dropTemporaryView(path: String): Boolean = ??? + }