Interface Table
-
- All Superinterfaces:
Executable
,Explainable<Table>
- All Known Implementing Classes:
TableImpl
@PublicEvolving public interface Table extends Explainable<Table>, Executable
TheTable
object is the core abstraction of the Table API. Similar to how the DataStream API hasDataStream
s, the Table API is built aroundTable
s.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 aDynamicTableSource
and how to eventually write data to aDynamicTableSink
. 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 aTableEnvironment
. For example,TableEnvironment.from(String)
) obtains a table from a catalog. EveryTable
object has a schema that is available throughgetResolvedSchema()
. ATable
object is always associated with its original table environment during programming.Every transformation (i.e.
select(Expression...)
orfilter(Expression)
) on aTable
object leads to a newTable
object.Use
Executable.execute()
to execute the pipeline and retrieve the transformed data locally during development. Otherwise, useexecuteInsert(String)
to write the data into a table sink.Many methods of this class take one or more
Expression
s 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
All Methods Instance Methods Abstract Methods Default Methods Deprecated Methods Modifier and Type Method Description Table
addColumns(Expression... fields)
Adds additional columns.Table
addOrReplaceColumns(Expression... fields)
Adds additional columns.AggregatedTable
aggregate(Expression aggregateFunction)
Performs a global aggregate operation with an aggregate function.Table
as(String field, String... fields)
Renames the fields of the expression result.Table
as(Expression... fields)
Deprecated.TemporalTableFunction
createTemporalTableFunction(Expression timeAttribute, Expression primaryKey)
CreatesTemporalTableFunction
backed up by this table as a history table.Table
distinct()
Removes duplicate values and returns only distinct (different) values.Table
dropColumns(Expression... fields)
Drops existing columns.default TableResult
executeInsert(String tablePath)
Shorthand fortableEnv.insertInto(tablePath).execute()
.default TableResult
executeInsert(String tablePath, boolean overwrite)
Shorthand fortableEnv.insertInto(tablePath, overwrite).execute()
.default TableResult
executeInsert(TableDescriptor descriptor)
Shorthand fortableEnv.insertInto(descriptor).execute()
.default TableResult
executeInsert(TableDescriptor descriptor, boolean overwrite)
Shorthand fortableEnv.insertInto(descriptor, overwrite).execute()
.Table
fetch(int fetch)
Limits a (possibly sorted) result to the first n rows.Table
filter(Expression predicate)
Filters out elements that don't pass the filter predicate.FlatAggregateTable
flatAggregate(Expression tableAggregateFunction)
Perform a global flatAggregate without groupBy.Table
flatMap(Expression tableFunction)
Performs a flatMap operation with an user-defined table function or built-in table function.Table
fullOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s.QueryOperation
getQueryOperation()
Returns underlying logical representation of this table.ResolvedSchema
getResolvedSchema()
Returns the resolved schema of this table.default TableSchema
getSchema()
Deprecated.This method has been deprecated as part of FLIP-164.GroupedTable
groupBy(Expression... fields)
Groups the elements on some grouping keys.TablePipeline
insertInto(String tablePath)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) that was registered under the specified path.TablePipeline
insertInto(String tablePath, boolean overwrite)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) that was registered under the specified path.TablePipeline
insertInto(TableDescriptor descriptor)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) expressed via the givenTableDescriptor
.TablePipeline
insertInto(TableDescriptor descriptor, boolean overwrite)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) expressed via the givenTableDescriptor
.Table
intersect(Table right)
Intersects twoTable
s with duplicate records removed.Table
intersectAll(Table right)
Intersects twoTable
s.Table
join(Table right)
Joins twoTable
s.Table
join(Table right, Expression joinPredicate)
Joins twoTable
s.Table
joinLateral(Expression tableFunctionCall)
Joins thisTable
with an user-definedTableFunction
.Table
joinLateral(Expression tableFunctionCall, Expression joinPredicate)
Joins thisTable
with an user-definedTableFunction
.Table
leftOuterJoin(Table right)
Joins twoTable
s.Table
leftOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s.Table
leftOuterJoinLateral(Expression tableFunctionCall)
Joins thisTable
with an user-definedTableFunction
.Table
leftOuterJoinLateral(Expression tableFunctionCall, Expression joinPredicate)
Joins thisTable
with an user-definedTableFunction
.default Table
limit(int fetch)
Limits a (possibly sorted) result to the first n rows.default Table
limit(int offset, int fetch)
Limits a (possibly sorted) result to the first n rows from an offset position.Table
map(Expression mapFunction)
Performs a map operation with an user-defined scalar function or built-in scalar function.Table
minus(Table right)
Minus of twoTable
s with duplicate records removed.Table
minusAll(Table right)
Minus of twoTable
s.Table
offset(int offset)
Limits a (possibly sorted) result from an offset position.Table
orderBy(Expression... fields)
Sorts the givenTable
.void
printSchema()
Prints the schema of this table to the console in a summary format.Table
renameColumns(Expression... fields)
Renames existing columns.Table
rightOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s.Table
select(Expression... fields)
Performs a selection operation.Table
union(Table right)
Unions twoTable
s with duplicate records removed.Table
unionAll(Table right)
Unions twoTable
s.Table
where(Expression predicate)
Filters out elements that don't pass the filter predicate.GroupWindowedTable
window(GroupWindow groupWindow)
Groups the records of a table by assigning them to windows defined by a time or row interval.OverWindowedTable
window(OverWindow... overWindows)
Defines over-windows on the records of a table.-
Methods inherited from interface org.apache.flink.table.api.Executable
execute
-
Methods inherited from interface org.apache.flink.table.api.Explainable
explain, explain, printExplain
-
-
-
-
Method Detail
-
getSchema
@Deprecated default TableSchema getSchema()
Deprecated.This method has been deprecated as part of FLIP-164.TableSchema
has been replaced by two more dedicated classesSchema
andResolvedSchema
. UseSchema
for declaration in APIs.ResolvedSchema
is offered by the framework after resolution and validation.Returns the schema of this table.
-
getResolvedSchema
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
Table select(Expression... fields)
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
TemporalTableFunction createTemporalTableFunction(Expression timeAttribute, Expression primaryKey)
CreatesTemporalTableFunction
backed 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.TemporalTableFunction
provides a way how to access those data.For more information please check Flink's documentation on Temporal Tables.
Currently
TemporalTableFunction
s 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:
TemporalTableFunction
which is an instance ofTableFunction
. It takes one single argument, thetimeAttribute
, for which it returns matching version of theTable
, from whichTemporalTableFunction
was created.
-
as
Table as(String field, String... fields)
Renames the fields of the expression result. Use this to disambiguate fields before joining to operations.Example:
tab.as("a", "b")
-
as
@Deprecated Table as(Expression... fields)
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
Table filter(Expression predicate)
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
Table where(Expression predicate)
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
GroupedTable groupBy(Expression... fields)
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
Table join(Table right)
Joins twoTable
s. Similar to a SQL join. The fields of the two joined operations must not overlap, useas
to 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
Table join(Table right, Expression joinPredicate)
Joins twoTable
s. Similar to a SQL join. The fields of the two joined operations must not overlap, useas
to 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
Table leftOuterJoin(Table right)
Joins twoTable
s. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, useas
to rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironment
and itsTableConfig
must have null check enabled (default).Example:
left.leftOuterJoin(right) .select($("a"), $("b"), $("d"));
-
leftOuterJoin
Table leftOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, useas
to rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironment
and itsTableConfig
must 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
Table rightOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s. Similar to a SQL right outer join. The fields of the two joined operations must not overlap, useas
to rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironment
and itsTableConfig
must 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
Table fullOuterJoin(Table right, Expression joinPredicate)
Joins twoTable
s. Similar to a SQL full outer join. The fields of the two joined operations must not overlap, useas
to rename fields if necessary.Note: Both tables must be bound to the same
TableEnvironment
and itsTableConfig
must 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
Table joinLateral(Expression tableFunctionCall)
Joins thisTable
with 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(Expression tableFunctionCall, Expression joinPredicate)
Joins thisTable
with 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
Table leftOuterJoinLateral(Expression tableFunctionCall)
Joins thisTable
with 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(Expression tableFunctionCall, Expression joinPredicate)
Joins thisTable
with 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
Table minus(Table right)
Minus of twoTable
s 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
Table minusAll(Table right)
Minus of twoTable
s. 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
Table union(Table right)
Unions twoTable
s 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
Table unionAll(Table right)
Unions twoTable
s. 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
Table intersect(Table right)
Intersects twoTable
s 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
Table intersectAll(Table right)
Intersects twoTable
s. 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
Table orderBy(Expression... fields)
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
Table offset(int 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
Table fetch(int 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
default Table limit(int fetch)
Limits a (possibly sorted) result to the first n rows.This method is a synonym for
fetch(int)
.
-
limit
default Table limit(int offset, int fetch)
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
GroupWindowedTable window(GroupWindow groupWindow)
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
OverWindowedTable window(OverWindow... overWindows)
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
Table addColumns(Expression... fields)
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
Table addOrReplaceColumns(Expression... fields)
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
Table renameColumns(Expression... fields)
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
Table dropColumns(Expression... fields)
Drops existing columns. The field expressions should be field reference expressions.Java Example:
tab.dropColumns($("a"), $("b"));
Scala Example:
tab.dropColumns($"a", $"b")
-
map
Table map(Expression mapFunction)
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
Table flatMap(Expression tableFunction)
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
AggregatedTable aggregate(Expression aggregateFunction)
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(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
TablePipeline insertInto(String tablePath)
Declares that the pipeline defined by the givenTable
object 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
TablePipeline
usingExecutable.execute()
, or compile it to aCompiledPlan
usingCompilable.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
TablePipeline insertInto(String tablePath, boolean overwrite)
Declares that the pipeline defined by the givenTable
object 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
TablePipeline
usingExecutable.execute()
, or compile it to aCompiledPlan
usingCompilable.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
TablePipeline insertInto(TableDescriptor descriptor)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) expressed via the givenTableDescriptor
.The
descriptor
won'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
Schema
for the sink descriptor. The declaration is similar to aCREATE TABLE
DDL 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
TablePipeline
usingExecutable.execute()
, or compile it to aCompiledPlan
usingCompilable.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
TablePipeline insertInto(TableDescriptor descriptor, boolean overwrite)
Declares that the pipeline defined by the givenTable
object should be written to a table (backed by aDynamicTableSink
) expressed via the givenTableDescriptor
.The
descriptor
won'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
Schema
for the sink descriptor. The declaration is similar to aCREATE TABLE
DDL 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
TablePipeline
usingExecutable.execute()
, or compile it to aCompiledPlan
usingCompilable.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
default TableResult executeInsert(String tablePath)
Shorthand fortableEnv.insertInto(tablePath).execute()
.- See Also:
insertInto(String)
,Executable.execute()
-
executeInsert
default TableResult executeInsert(String tablePath, boolean overwrite)
Shorthand fortableEnv.insertInto(tablePath, overwrite).execute()
.- See Also:
insertInto(String, boolean)
,Executable.execute()
-
executeInsert
default TableResult executeInsert(TableDescriptor descriptor)
Shorthand fortableEnv.insertInto(descriptor).execute()
.- See Also:
insertInto(TableDescriptor)
,Executable.execute()
-
executeInsert
default TableResult executeInsert(TableDescriptor descriptor, boolean overwrite)
Shorthand fortableEnv.insertInto(descriptor, overwrite).execute()
.
-
-