Interface Table
- All Superinterfaces:
Executable,Explainable<Table>
- All Known Implementing Classes:
TableImpl
Table object is the core abstraction of the Table API. Similar to how the DataStream
API has DataStreams, the Table API is built around Tables.
A Table object describes a pipeline of data transformations. It does not contain the
data itself in any way. Instead, it describes how to read data from a DynamicTableSource
and how to eventually write data to a DynamicTableSink. The declared pipeline can be
printed, optimized, and eventually executed in a cluster. The pipeline can work with bounded or
unbounded streams which enables both streaming and batch scenarios.
By the definition above, a Table object can actually be considered as a view in
SQL terms.
The initial Table object is constructed by a TableEnvironment. For example,
TableEnvironment.from(String)) obtains a table from a catalog. Every Table object
has a schema that is available through getResolvedSchema(). A Table object is
always associated with its original table environment during programming.
Every transformation (i.e. select(Expression...) or filter(Expression)) on a
Table object leads to a new Table object.
Use Executable.execute() to execute the pipeline and retrieve the transformed data locally
during development. Otherwise, use executeInsert(String) to write the data into a table
sink.
Many methods of this class take one or more Expressions as parameters. For fluent
definition of expressions and easier readability, we recommend to add a star import:
import static org.apache.flink.table.api.Expressions.*;
Check the documentation for more programming language specific APIs, for example, by using Scala implicits.
The following example shows how to work with a Table object.
Java Example (with static import for expressions):
TableEnvironment tableEnv = TableEnvironment.create(...);
Table table = tableEnv.from("MyTable").select($("colA").trim(), $("colB").plus(12));
table.execute().print();
Scala Example (with implicits for expressions):
val tableEnv = TableEnvironment.create(...)
val table = tableEnv.from("MyTable").select($"colA".trim(), $"colB" + 12)
table.execute().print()
-
Method Summary
Modifier and TypeMethodDescriptionaddColumns(org.apache.flink.table.expressions.Expression... fields) Adds additional columns.addOrReplaceColumns(org.apache.flink.table.expressions.Expression... fields) Adds additional columns.aggregate(org.apache.flink.table.expressions.Expression aggregateFunction) Performs a global aggregate operation with an aggregate function.Renames the fields of the expression result.as(org.apache.flink.table.expressions.Expression... fields) Deprecated.org.apache.flink.table.functions.TemporalTableFunctioncreateTemporalTableFunction(org.apache.flink.table.expressions.Expression timeAttribute, org.apache.flink.table.expressions.Expression primaryKey) CreatesTemporalTableFunctionbacked up by this table as a history table.distinct()Removes duplicate values and returns only distinct (different) values.dropColumns(org.apache.flink.table.expressions.Expression... fields) Drops existing columns.default TableResultexecuteInsert(String tablePath) Shorthand fortableEnv.insertInto(tablePath).execute().default TableResultexecuteInsert(String tablePath, boolean overwrite) Shorthand fortableEnv.insertInto(tablePath, overwrite).execute().default TableResultexecuteInsert(TableDescriptor descriptor) Shorthand fortableEnv.insertInto(descriptor).execute().default TableResultexecuteInsert(TableDescriptor descriptor, boolean overwrite) Shorthand fortableEnv.insertInto(descriptor, overwrite).execute().fetch(int fetch) Limits a (possibly sorted) result to the first n rows.filter(org.apache.flink.table.expressions.Expression predicate) Filters out elements that don't pass the filter predicate.flatAggregate(org.apache.flink.table.expressions.Expression tableAggregateFunction) Perform a global flatAggregate without groupBy.flatMap(org.apache.flink.table.expressions.Expression tableFunction) Performs a flatMap operation with an user-defined table function or built-in table function.fullOuterJoin(Table right, org.apache.flink.table.expressions.Expression joinPredicate) Joins twoTables.Returns underlying logical representation of this table.org.apache.flink.table.catalog.ResolvedSchemaReturns the resolved schema of this table.default org.apache.flink.table.legacy.api.TableSchemaDeprecated.This method has been deprecated as part of FLIP-164.groupBy(org.apache.flink.table.expressions.Expression... fields) Groups the elements on some grouping keys.insertInto(String tablePath) Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) that was registered under the specified path.insertInto(String tablePath, boolean overwrite) Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) that was registered under the specified path.insertInto(TableDescriptor descriptor) Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) expressed via the givenTableDescriptor.insertInto(TableDescriptor descriptor, boolean overwrite) Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) expressed via the givenTableDescriptor.Intersects twoTables with duplicate records removed.intersectAll(Table right) Intersects twoTables.Joins twoTables.Joins twoTables.joinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall) Joins thisTablewith an user-definedTableFunction.joinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall, org.apache.flink.table.expressions.Expression joinPredicate) Joins thisTablewith an user-definedTableFunction.leftOuterJoin(Table right) Joins twoTables.leftOuterJoin(Table right, org.apache.flink.table.expressions.Expression joinPredicate) Joins twoTables.leftOuterJoinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall) Joins thisTablewith an user-definedTableFunction.leftOuterJoinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall, org.apache.flink.table.expressions.Expression joinPredicate) Joins thisTablewith an user-definedTableFunction.default Tablelimit(int fetch) Limits a (possibly sorted) result to the first n rows.default Tablelimit(int offset, int fetch) Limits a (possibly sorted) result to the first n rows from an offset position.map(org.apache.flink.table.expressions.Expression mapFunction) Performs a map operation with an user-defined scalar function or built-in scalar function.Minus of twoTables with duplicate records removed.Minus of twoTables.offset(int offset) Limits a (possibly sorted) result from an offset position.orderBy(org.apache.flink.table.expressions.Expression... fields) Sorts the givenTable.voidPrints the schema of this table to the console in a summary format.renameColumns(org.apache.flink.table.expressions.Expression... fields) Renames existing columns.rightOuterJoin(Table right, org.apache.flink.table.expressions.Expression joinPredicate) Joins twoTables.select(org.apache.flink.table.expressions.Expression... fields) Performs a selection operation.Unions twoTables with duplicate records removed.Unions twoTables.where(org.apache.flink.table.expressions.Expression predicate) Filters out elements that don't pass the filter predicate.window(GroupWindow groupWindow) Groups the records of a table by assigning them to windows defined by a time or row interval.window(OverWindow... overWindows) Defines over-windows on the records of a table.Methods inherited from interface org.apache.flink.table.api.Executable
executeMethods inherited from interface org.apache.flink.table.api.Explainable
explain, explain, printExplain
-
Method Details
-
getSchema
Deprecated.This method has been deprecated as part of FLIP-164.TableSchemahas been replaced by two more dedicated classesSchemaandResolvedSchema. UseSchemafor declaration in APIs.ResolvedSchemais offered by the framework after resolution and validation.Returns the schema of this table. -
getResolvedSchema
org.apache.flink.table.catalog.ResolvedSchema getResolvedSchema()Returns the resolved schema of this table. -
printSchema
void printSchema()Prints the schema of this table to the console in a summary format. -
getQueryOperation
QueryOperation getQueryOperation()Returns underlying logical representation of this table. -
select
Performs a selection operation. Similar to a SQL SELECT statement. The field expressions can contain complex expressions and aggregations.Java Example:
tab.select($("key"), $("value").avg().plus(" The average").as("average"));Scala Example:
tab.select($"key", $"value".avg + " The average" as "average") -
createTemporalTableFunction
org.apache.flink.table.functions.TemporalTableFunction createTemporalTableFunction(org.apache.flink.table.expressions.Expression timeAttribute, org.apache.flink.table.expressions.Expression primaryKey) CreatesTemporalTableFunctionbacked up by this table as a history table. Temporal Tables represent a concept of a table that changes over time and for which Flink keeps track of those changes.TemporalTableFunctionprovides a way how to access those data.For more information please check Flink's documentation on Temporal Tables.
Currently
TemporalTableFunctions are only supported in streaming.- Parameters:
timeAttribute- Must points to a time indicator. Provides a way to compare which records are a newer or older version.primaryKey- Defines the primary key. With primary key it is possible to update a row or to delete it.- Returns:
TemporalTableFunctionwhich is an instance ofTableFunction. It takes one single argument, thetimeAttribute, for which it returns matching version of theTable, from whichTemporalTableFunctionwas created.
-
as
Renames the fields of the expression result. Use this to disambiguate fields before joining to operations.Example:
tab.as("a", "b") -
as
Deprecated.Renames the fields of the expression result. Use this to disambiguate fields before joining to operations.Java Example:
tab.as($("a"), $("b"))Scala Example:
tab.as($"a", $"b") -
filter
Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE clause.Java Example:
tab.filter($("name").isEqual("Fred"));Scala Example:
tab.filter($"name" === "Fred") -
where
Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE clause.Java Example:
tab.where($("name").isEqual("Fred"));Scala Example:
tab.where($"name" === "Fred") -
groupBy
Groups the elements on some grouping keys. Use this before a selection with aggregations to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement.Java Example:
tab.groupBy($("key")).select($("key"), $("value").avg());Scala Example:
tab.groupBy($"key").select($"key", $"value".avg) -
distinct
Table distinct()Removes duplicate values and returns only distinct (different) values.Example:
tab.select($("key"), $("value")).distinct(); -
join
Joins twoTables. Similar to a SQL join. The fields of the two joined operations must not overlap, useasto rename fields if necessary. You can use where and select clauses after a join to further specify the behaviour of the join.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.join(right) .where($("a").isEqual($("b")).and($("c").isGreater(3)) .select($("a"), $("b"), $("d")); -
join
Joins twoTables. Similar to a SQL join. The fields of the two joined operations must not overlap, useasto rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironment.Java Example:
left.join(right, $("a").isEqual($("b"))) .select($("a"), $("b"), $("d"));Scala Example:
left.join(right, $"a" === $"b") .select($"a", $"b", $"d") -
leftOuterJoin
Joins twoTables. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, useasto rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironmentand itsTableConfigmust have null check enabled (default).Example:
left.leftOuterJoin(right) .select($("a"), $("b"), $("d")); -
leftOuterJoin
Joins twoTables. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, useasto rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironmentand itsTableConfigmust have null check enabled (default).Java Example:
left.leftOuterJoin(right, $("a").isEqual($("b"))) .select($("a"), $("b"), $("d"));Scala Example:
left.leftOuterJoin(right, $"a" === $"b") .select($"a", $"b", $"d") -
rightOuterJoin
Joins twoTables. Similar to a SQL right outer join. The fields of the two joined operations must not overlap, useasto rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironmentand itsTableConfigmust have null check enabled (default).Java Example:
left.rightOuterJoin(right, $("a").isEqual($("b"))) .select($("a"), $("b"), $("d"));Scala Example:
left.rightOuterJoin(right, $"a" === $"b") .select($"a", $"b", $"d") -
fullOuterJoin
Joins twoTables. Similar to a SQL full outer join. The fields of the two joined operations must not overlap, useasto rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironmentand itsTableConfigmust have null check enabled (default).Java Example:
left.fullOuterJoin(right, $("a").isEqual($("b"))) .select($("a"), $("b"), $("d"));Scala Example:
left.fullOuterJoin(right, $"a" === $"b") .select($"a", $"b", $"d") -
joinLateral
Joins thisTablewith an user-definedTableFunction. This join is similar to a SQL inner join with ON TRUE predicate but works with a table function. Each row of the table is joined with all rows produced by the table function.Java Example:
class MySplitUDTF extends TableFunction<String> { public void eval(String str) { str.split("#").forEach(this::collect); } } table.joinLateral(call(MySplitUDTF.class, $("c")).as("s")) .select($("a"), $("b"), $("c"), $("s"));Scala Example:
class MySplitUDTF extends TableFunction[String] { def eval(str: String): Unit = { str.split("#").foreach(collect) } } val split = new MySplitUDTF() table.joinLateral(split($"c") as "s") .select($"a", $"b", $"c", $"s") -
joinLateral
Table joinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall, org.apache.flink.table.expressions.Expression joinPredicate) Joins thisTablewith an user-definedTableFunction. This join is similar to a SQL inner join but works with a table function. Each row of the table is joined with all rows produced by the table function.Java Example:
class MySplitUDTF extends TableFunction<String> { public void eval(String str) { str.split("#").forEach(this::collect); } } table.joinLateral(call(MySplitUDTF.class, $("c")).as("s"), $("a").isEqual($("s"))) .select($("a"), $("b"), $("c"), $("s"));Scala Example:
class MySplitUDTF extends TableFunction[String] { def eval(str: String): Unit = { str.split("#").foreach(collect) } } val split = new MySplitUDTF() table.joinLateral(split($"c") as "s", $"a" === $"s") .select($"a", $"b", $"c", $"s") -
leftOuterJoinLateral
Joins thisTablewith an user-definedTableFunction. This join is similar to a SQL left outer join with ON TRUE predicate but works with a table function. Each row of the table is joined with all rows produced by the table function. If the table function does not produce any row, the outer row is padded with nulls.Java Example:
class MySplitUDTF extends TableFunction<String> { public void eval(String str) { str.split("#").forEach(this::collect); } } table.leftOuterJoinLateral(call(MySplitUDTF.class, $("c")).as("s")) .select($("a"), $("b"), $("c"), $("s"));Scala Example:
class MySplitUDTF extends TableFunction[String] { def eval(str: String): Unit = { str.split("#").foreach(collect) } } val split = new MySplitUDTF() table.leftOuterJoinLateral(split($"c") as "s") .select($"a", $"b", $"c", $"s") -
leftOuterJoinLateral
Table leftOuterJoinLateral(org.apache.flink.table.expressions.Expression tableFunctionCall, org.apache.flink.table.expressions.Expression joinPredicate) Joins thisTablewith an user-definedTableFunction. This join is similar to a SQL left outer join with ON TRUE predicate but works with a table function. Each row of the table is joined with all rows produced by the table function. If the table function does not produce any row, the outer row is padded with nulls.Java Example:
class MySplitUDTF extends TableFunction<String> { public void eval(String str) { str.split("#").forEach(this::collect); } } table.leftOuterJoinLateral(call(MySplitUDTF.class, $("c")).as("s"), $("a").isEqual($("s"))) .select($("a"), $("b"), $("c"), $("s"));Scala Example:
class MySplitUDTF extends TableFunction[String] { def eval(str: String): Unit = { str.split("#").foreach(collect) } } val split = new MySplitUDTF() table.leftOuterJoinLateral(split($"c") as "s", $"a" === $"s") .select($"a", $"b", $"c", $"s") -
minus
Minus of twoTables with duplicate records removed. Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not exist in the right table. Duplicate records in the left table are returned exactly once, i.e., duplicates are removed. Both tables must have identical field types.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.minus(right); -
minusAll
Minus of twoTables. Similar to a SQL EXCEPT ALL. Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in the right table. A record that is present n times in the left table and m times in the right table is returned (n - m) times, i.e., as many duplicates as are present in the right table are removed. Both tables must have identical field types.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.minusAll(right); -
union
Unions twoTables with duplicate records removed. Similar to a SQL UNION. The fields of the two union operations must fully overlap.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.union(right); -
unionAll
Unions twoTables. Similar to a SQL UNION ALL. The fields of the two union operations must fully overlap.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.unionAll(right); -
intersect
Intersects twoTables with duplicate records removed. Intersect returns records that exist in both tables. If a record is present in one or both tables more than once, it is returned just once, i.e., the resulting table has no duplicate records. Similar to a SQL INTERSECT. The fields of the two intersect operations must fully overlap.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.intersect(right); -
intersectAll
Intersects twoTables. IntersectAll returns records that exist in both tables. If a record is present in both tables more than once, it is returned as many times as it is present in both tables, i.e., the resulting table might have duplicate records. Similar to an SQL INTERSECT ALL. The fields of the two intersect operations must fully overlap.Note: Both tables must be bound to the same
TableEnvironment.Example:
left.intersectAll(right); -
orderBy
Sorts the givenTable. Similar to SQLORDER BY.The resulting Table is globally sorted across all parallel partitions.
Java Example:
tab.orderBy($("name").desc());Scala Example:
tab.orderBy($"name".desc)For unbounded tables, this operation requires a sorting on a time attribute or a subsequent fetch operation.
-
offset
Limits a (possibly sorted) result from an offset position.This method can be combined with a preceding
orderBy(Expression...)call for a deterministic order and a subsequentfetch(int)call to return n rows after skipping the first o rows.// skips the first 3 rows and returns all following rows. tab.orderBy($("name").desc()).offset(3); // skips the first 10 rows and returns the next 5 rows. tab.orderBy($("name").desc()).offset(10).fetch(5);For unbounded tables, this operation requires a subsequent fetch operation.
- Parameters:
offset- number of records to skip
-
fetch
Limits a (possibly sorted) result to the first n rows.This method can be combined with a preceding
orderBy(Expression...)call for a deterministic order andoffset(int)call to return n rows after skipping the first o rows.// returns the first 3 records. tab.orderBy($("name").desc()).fetch(3); // skips the first 10 rows and returns the next 5 rows. tab.orderBy($("name").desc()).offset(10).fetch(5);- Parameters:
fetch- the number of records to return. Fetch must be >= 0.
-
limit
Limits a (possibly sorted) result to the first n rows.This method is a synonym for
fetch(int). -
limit
Limits a (possibly sorted) result to the first n rows from an offset position.This method is a synonym for
offset(int)followed byfetch(int). -
window
Groups the records of a table by assigning them to windows defined by a time or row interval.For streaming tables of infinite size, grouping into windows is required to define finite groups on which group-based aggregates can be computed.
For batch tables of finite size, windowing essentially provides shortcuts for time-based groupBy.
Note: Computing windowed aggregates on a streaming table is only a parallel operation if additional grouping attributes are added to the
groupBy(...)clause. If thegroupBy(...)only references a GroupWindow alias, the streamed table will be processed by a single task, i.e., with parallelism 1.- Parameters:
groupWindow- groupWindow that specifies how elements are grouped.- Returns:
- A windowed table.
-
window
Defines over-windows on the records of a table.An over-window defines for each record an interval of records over which aggregation functions can be computed.
Java Example:
table .window(Over.partitionBy($("c")).orderBy($("rowTime")).preceding(lit(10).seconds()).as("ow") .select($("c"), $("b").count().over($("ow")), $("e").sum().over($("ow")));Scala Example:
table .window(Over partitionBy $"c" orderBy $"rowTime" preceding 10.seconds as "ow") .select($"c", $"b".count over $"ow", $"e".sum over $"ow")Note: Computing over window aggregates on a streaming table is only a parallel operation if the window is partitioned. Otherwise, the whole stream will be processed by a single task, i.e., with parallelism 1.
Note: Over-windows for batch tables are currently not supported.
- Parameters:
overWindows- windows that specify the record interval over which aggregations are computed.- Returns:
- An OverWindowedTable to specify the aggregations.
-
addColumns
Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. It will throw an exception if the added fields already exist.Java Example:
tab.addColumns( $("a").plus(1).as("a1"), concat($("b"), "sunny").as("b1") );Scala Example:
tab.addColumns( $"a" + 1 as "a1", concat($"b", "sunny") as "b1" ) -
addOrReplaceColumns
Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. Existing fields will be replaced. If the added fields have duplicate field name, then the last one is used.Java Example:
tab.addOrReplaceColumns( $("a").plus(1).as("a1"), concat($("b"), "sunny").as("b1") );Scala Example:
tab.addOrReplaceColumns( $"a" + 1 as "a1", concat($"b", "sunny") as "b1" ) -
renameColumns
Renames existing columns. Similar to a field alias statement. The field expressions should be alias expressions, and only the existing fields can be renamed.Java Example:
tab.renameColumns( $("a").as("a1"), $("b").as("b1") );Scala Example:
tab.renameColumns( $"a" as "a1", $"b" as "b1" ) -
dropColumns
Drops existing columns. The field expressions should be field reference expressions.Java Example:
tab.dropColumns($("a"), $("b"));Scala Example:
tab.dropColumns($"a", $"b") -
map
Performs a map operation with an user-defined scalar function or built-in scalar function. The output will be flattened if the output type is a composite type.Java Example:
tab.map(call(MyMapFunction.class, $("c")))Scala Example:
val func = new MyMapFunction() tab.map(func($"c")) -
flatMap
Performs a flatMap operation with an user-defined table function or built-in table function. The output will be flattened if the output type is a composite type.Java Example:
tab.flatMap(call(MyFlatMapFunction.class, $("c")))Scala Example:
val func = new MyFlatMapFunction() tab.flatMap(func($"c")) -
aggregate
Performs a global aggregate operation with an aggregate function. You have to close theaggregate(Expression)with a select statement. The output will be flattened if the output type is a composite type.Java Example:
tab.aggregate(call(MyAggregateFunction.class, $("a"), $("b")).as("f0", "f1", "f2")) .select($("f0"), $("f1"));Scala Example:
val aggFunc = new MyAggregateFunction table.aggregate(aggFunc($"a", $"b") as ("f0", "f1", "f2")) .select($"f0", $"f1") -
flatAggregate
FlatAggregateTable flatAggregate(org.apache.flink.table.expressions.Expression tableAggregateFunction) Perform a global flatAggregate without groupBy. FlatAggregate takes a TableAggregateFunction which returns multiple rows. Use a selection after the flatAggregate.Java Example:
tab.flatAggregate(call(MyTableAggregateFunction.class, $("a"), $("b")).as("x", "y", "z")) .select($("x"), $("y"), $("z"));Scala Example:
val tableAggFunc: TableAggregateFunction = new MyTableAggregateFunction tab.flatAggregate(tableAggFunc($"a", $"b") as ("x", "y", "z")) .select($"x", $"y", $"z") -
insertInto
Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) that was registered under the specified path.See the documentation of
TableEnvironment.useDatabase(String)orTableEnvironment.useCatalog(String)for the rules on the path resolution.Example:
Table table = tableEnv.sqlQuery("SELECT * FROM MyTable"); TablePipeline tablePipeline = table.insertInto("MySinkTable"); TableResult tableResult = tablePipeline.execute(); tableResult.await();One can execute the returned
TablePipelineusingExecutable.execute(), or compile it to aCompiledPlanusingCompilable.compilePlan().If multiple pipelines should insert data into one or more sink tables as part of a single execution, use a
StatementSet(seeTableEnvironment.createStatementSet()).- Parameters:
tablePath- The path of the registered table (backed by aDynamicTableSink).- Returns:
- The complete pipeline from one or more source tables to a sink table.
-
insertInto
Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) that was registered under the specified path.See the documentation of
TableEnvironment.useDatabase(String)orTableEnvironment.useCatalog(String)for the rules on the path resolution.Example:
Table table = tableEnv.sqlQuery("SELECT * FROM MyTable"); TablePipeline tablePipeline = table.insertInto("MySinkTable", true); TableResult tableResult = tablePipeline.execute(); tableResult.await();One can execute the returned
TablePipelineusingExecutable.execute(), or compile it to aCompiledPlanusingCompilable.compilePlan().If multiple pipelines should insert data into one or more sink tables as part of a single execution, use a
StatementSet(seeTableEnvironment.createStatementSet()).- Parameters:
tablePath- The path of the registered table (backed by aDynamicTableSink).overwrite- Indicates whether existing data should be overwritten.- Returns:
- The complete pipeline from one or more source tables to a sink table.
-
insertInto
Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) expressed via the givenTableDescriptor.The
descriptorwon't be registered in the catalog, but it will be propagated directly in the operation tree. Note that calling this method multiple times, even with the same descriptor, results in multiple sink tables instances.This method allows to declare a
Schemafor the sink descriptor. The declaration is similar to aCREATE TABLEDDL in SQL and allows to:- overwrite automatically derived columns with a custom
DataType - add metadata columns next to the physical columns
- declare a primary key
It is possible to declare a schema without physical/regular columns. In this case, those columns will be automatically derived and implicitly put at the beginning of the schema declaration.
Examples:
Schema schema = Schema.newBuilder() .column("f0", DataTypes.STRING()) .build(); Table table = tableEnv.from(TableDescriptor.forConnector("datagen") .schema(schema) .build()); table.insertInto(TableDescriptor.forConnector("blackhole") .schema(schema) .build());One can execute the returned
TablePipelineusingExecutable.execute(), or compile it to aCompiledPlanusingCompilable.compilePlan().If multiple pipelines should insert data into one or more sink tables as part of a single execution, use a
StatementSet(seeTableEnvironment.createStatementSet()).- Parameters:
descriptor- Descriptor describing the sink table into which data should be inserted.- Returns:
- The complete pipeline from one or more source tables to a sink table.
- overwrite automatically derived columns with a custom
-
insertInto
Declares that the pipeline defined by the givenTableobject should be written to a table (backed by aDynamicTableSink) expressed via the givenTableDescriptor.The
descriptorwon't be registered in the catalog, but it will be propagated directly in the operation tree. Note that calling this method multiple times, even with the same descriptor, results in multiple sink tables being registered.This method allows to declare a
Schemafor the sink descriptor. The declaration is similar to aCREATE TABLEDDL in SQL and allows to:- overwrite automatically derived columns with a custom
DataType - add metadata columns next to the physical columns
- declare a primary key
It is possible to declare a schema without physical/regular columns. In this case, those columns will be automatically derived and implicitly put at the beginning of the schema declaration.
Examples:
Schema schema = Schema.newBuilder() .column("f0", DataTypes.STRING()) .build(); Table table = tableEnv.from(TableDescriptor.forConnector("datagen") .schema(schema) .build()); table.insertInto(TableDescriptor.forConnector("blackhole") .schema(schema) .build(), true);One can execute the returned
TablePipelineusingExecutable.execute(), or compile it to aCompiledPlanusingCompilable.compilePlan().If multiple pipelines should insert data into one or more sink tables as part of a single execution, use a
StatementSet(seeTableEnvironment.createStatementSet()).- Parameters:
descriptor- Descriptor describing the sink table into which data should be inserted.overwrite- Indicates whether existing data should be overwritten.- Returns:
- The complete pipeline from one or more source tables to a sink table.
- overwrite automatically derived columns with a custom
-
executeInsert
Shorthand fortableEnv.insertInto(tablePath).execute().- See Also:
-
executeInsert
Shorthand fortableEnv.insertInto(tablePath, overwrite).execute(). -
executeInsert
Shorthand fortableEnv.insertInto(descriptor).execute(). -
executeInsert
Shorthand fortableEnv.insertInto(descriptor, overwrite).execute().
-
as(String, String...)