Skip to content

Commit

Permalink
[FLINK-14490][table-api] Rework insertInto method
Browse files Browse the repository at this point in the history
  • Loading branch information
dawidwys committed Oct 29, 2019
1 parent b286330 commit fe966d4
Show file tree
Hide file tree
Showing 36 changed files with 412 additions and 204 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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));
Expand Down Expand Up @@ -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<String> result = hiveShell.executeQuery("select * from " + tblName);
Expand Down Expand Up @@ -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<String> result = hiveShell.executeQuery("select * from " + tblName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,9 +133,7 @@ public static List<Row> 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<byte[]> data = result.getAccumulatorResult(id);
return SerializedListAccumulator.deserializeList(data, serializer);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -564,8 +563,7 @@ private List<Row> 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")));
}
}

Expand Down
13 changes: 4 additions & 9 deletions flink-python/pyflink/table/table.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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):
"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,13 @@ def excluded_methods(cls):
'getCompletionHints',
'create',
'loadModule',
'unloadModule'}
'unloadModule',
'listTemporaryTables',
'createTemporaryView',
'dropTemporaryTable',
'listTemporaryViews',
'from',
'dropTemporaryView'}


if __name__ == '__main__':
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -494,8 +493,6 @@ private <C> ResultDescriptor executeQueryInternal(ExecutionContext<C> context, S
envInst.getTableEnvironment().registerTableSink(jobName, result.getTableSink());
table.insertInto(
envInst.getQueryConfig(),
EnvironmentSettings.DEFAULT_BUILTIN_CATALOG,
EnvironmentSettings.DEFAULT_BUILTIN_DATABASE,
jobName);
return null;
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -309,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);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron
* .field("count", "DECIMAL")
* .field("proc-time", "TIMESTAMP").proctime())
* .inAppendMode()
* .registerSource("MyTable")
* .createTemporaryTable("MyTable")
* }
* </pre>
*
Expand Down Expand Up @@ -549,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);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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);
Expand All @@ -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);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,9 @@ static TableEnvironment create(EnvironmentSettings settings) {
*
* @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);

/**
Expand All @@ -191,7 +193,7 @@ static TableEnvironment create(EnvironmentSettings settings) {
* @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);
Expand All @@ -207,7 +209,9 @@ static TableEnvironment create(EnvironmentSettings settings) {
*
* @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);

/**
Expand Down Expand Up @@ -294,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.
*
* <p>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.
*
Expand Down
Original file line number Diff line number Diff line change
@@ -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);
}
Loading

0 comments on commit fe966d4

Please sign in to comment.