@PublicEvolving public interface BatchTableEnvironment extends TableEnvironment
TableEnvironment
for a Java batch ExecutionEnvironment
that works with DataSet
s.
A TableEnvironment can be used to:
DataSet
to a Table
DataSet
in the TableEnvironment
's catalog
Table
in the TableEnvironment
's catalog
Table
Table
Table
into a DataSet
Table
Modifier and Type | Method and Description |
---|---|
BatchTableDescriptor |
connect(ConnectorDescriptor connectorDescriptor)
Deprecated.
The SQL
CREATE TABLE DDL is richer than this part of the API. This method
might be refactored in the next versions. Please use executeSql(ddl) to register a table instead. |
static BatchTableEnvironment |
create(ExecutionEnvironment executionEnvironment)
|
static BatchTableEnvironment |
create(ExecutionEnvironment executionEnvironment,
TableConfig tableConfig)
|
<T> void |
createTemporaryView(String path,
DataSet<T> dataSet)
Creates a view from the given
DataSet in a given path. |
<T> void |
createTemporaryView(String path,
DataSet<T> dataSet,
Expression... fields)
Creates a view from the given
DataSet in a given path with specified field names. |
<T> void |
createTemporaryView(String path,
DataSet<T> dataSet,
String fields)
Deprecated.
|
<T> Table |
fromDataSet(DataSet<T> dataSet)
|
<T> Table |
fromDataSet(DataSet<T> dataSet,
Expression... fields)
|
<T> Table |
fromDataSet(DataSet<T> dataSet,
String fields)
Deprecated.
|
<T> void |
registerDataSet(String name,
DataSet<T> dataSet)
Deprecated.
|
<T> void |
registerDataSet(String name,
DataSet<T> dataSet,
String fields)
Deprecated.
|
<T,ACC> void |
registerFunction(String name,
AggregateFunction<T,ACC> aggregateFunction)
Registers an
AggregateFunction under a unique name in the TableEnvironment's catalog. |
<T> void |
registerFunction(String name,
TableFunction<T> tableFunction)
Registers a
TableFunction under a unique name in the TableEnvironment's catalog. |
<T> DataSet<T> |
toDataSet(Table table,
Class<T> clazz)
|
<T> DataSet<T> |
toDataSet(Table table,
TypeInformation<T> typeInfo)
|
create, createFunction, createFunction, createStatementSet, createTemporaryFunction, createTemporaryFunction, createTemporarySystemFunction, createTemporarySystemFunction, createTemporaryView, dropFunction, dropTemporaryFunction, dropTemporarySystemFunction, dropTemporaryTable, dropTemporaryView, execute, executeSql, explain, explain, explain, explainSql, from, fromTableSource, fromValues, fromValues, fromValues, fromValues, fromValues, fromValues, getCatalog, getCompletionHints, getConfig, getCurrentCatalog, getCurrentDatabase, insertInto, insertInto, listCatalogs, listDatabases, listFunctions, listModules, listTables, listTemporaryTables, listTemporaryViews, listUserDefinedFunctions, listViews, loadModule, registerCatalog, registerFunction, registerTable, scan, sqlQuery, sqlUpdate, unloadModule, useCatalog, useDatabase
<T> void registerFunction(String name, TableFunction<T> tableFunction)
TableFunction
under a unique name in the TableEnvironment's catalog.
Registered functions can be referenced in Table API and SQL queries.T
- The type of the output row.name
- The name under which the function is registered.tableFunction
- The TableFunction to register.<T,ACC> void registerFunction(String name, AggregateFunction<T,ACC> aggregateFunction)
AggregateFunction
under a unique name in the TableEnvironment's catalog.
Registered functions can be referenced in Table API and SQL queries.T
- The type of the output value.ACC
- The type of aggregate accumulator.name
- The name under which the function is registered.aggregateFunction
- The AggregateFunction to register.@Deprecated <T> Table fromDataSet(DataSet<T> dataSet, String fields)
fromDataSet(DataSet, Expression...)
DataSet
into a Table
with specified field names.
There are two modes for mapping original fields to the fields of the Table
:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataSet<Tuple2<String, Long>> set = ...
// use the original 'f0' field and give a better name to the 'f1' field
Table table = tableEnv.fromTable(set, "f0, f1 as name");
2. Reference input fields by position: In this mode, fields are simply renamed. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataSet<Tuple2<String, Long>> set = ...
// renames the original fields as 'a' and 'b'
Table table = tableEnv.fromDataSet(set, "a, b");
<T> Table fromDataSet(DataSet<T> dataSet, Expression... fields)
DataSet
into a Table
with specified field names.
There are two modes for mapping original fields to the fields of the Table
:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataSet<Tuple2<String, Long>> set = ...
Table table = tableEnv.fromDataSet(
set,
$("f1"), // reorder and use the original field
$("f0").as("name") // reorder and give the original field a better name
);
2. Reference input fields by position: In this mode, fields are simply renamed. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataSet<Tuple2<String, Long>> set = ...
Table table = tableEnv.fromDataSet(
set,
$("a"), // renames the first field to 'a'
$("b") // renames the second field to 'b'
);
@Deprecated <T> void registerDataSet(String name, DataSet<T> dataSet)
createTemporaryView(String, DataSet)
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(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.
<T> void createTemporaryView(String path, DataSet<T> dataSet)
DataSet
in a given path. Registered views 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.
T
- The type of the DataSet
.path
- The path under which the view is created. See also the TableEnvironment
class description for the format of the path.dataSet
- The DataSet
out of which to create the view.@Deprecated <T> void registerDataSet(String name, DataSet<T> dataSet, String fields)
createTemporaryView(String, DataSet, String)
DataSet
in a given path with specified field names.
Registered views can be referenced in SQL queries.
There are two modes for mapping original fields to the fields of the View:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataSet<Tuple2<String, Long>> set = ...
// use the original 'f0' field and give a better name to the 'f1' field
tableEnv.registerDataSet("myTable", set, "f0, f1 as name");
2. Reference input fields by position: In this mode, fields are simply renamed. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataSet<Tuple2<String, Long>> set = ...
// renames the original fields as 'a' and 'b'
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(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.
@Deprecated <T> void createTemporaryView(String path, DataSet<T> dataSet, String fields)
createTemporaryView(String, DataSet, Expression...)
DataSet
in a given path with specified field names.
Registered views can be referenced in SQL queries.
There are two modes for mapping original fields to the fields of the View:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataSet<Tuple2<String, Long>> set = ...
// use the original 'f0' field and give a better name to the 'f1' field
tableEnv.createTemporaryView("cat.db.myTable", set, "f0, f1 as name");
2. Reference input fields by position: In this mode, fields are simply renamed. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataSet<Tuple2<String, Long>> set = ...
// renames the original fields as 'a' and 'b'
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.
T
- The type of the DataSet
.path
- The path under which the view is created. See also the TableEnvironment
class description for the format of the path.dataSet
- The DataSet
out of which to create the view.fields
- The fields expressions to map original fields of the DataSet to the fields of
the View.<T> void createTemporaryView(String path, DataSet<T> dataSet, Expression... fields)
DataSet
in a given path with specified field names.
Registered views can be referenced in SQL queries.
There are two modes for mapping original fields to the fields of the View:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataSet<Tuple2<String, Long>> set = ...
tableEnv.createTemporaryView(
"cat.db.myTable",
set,
$("f1"), // reorder and use the original field
$("f0").as("name") // reorder and give the original field a better name
);
2. Reference input fields by position: In this mode, fields are simply renamed. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataSet<Tuple2<String, Long>> set = ...
tableEnv.createTemporaryView(
"cat.db.myTable",
set,
$("a"), // renames the first field to 'a'
$("b") // renames the second field to '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.
T
- The type of the DataSet
.path
- The path under which the view is created. See also the TableEnvironment
class description for the format of the path.dataSet
- The DataSet
out of which to create the view.fields
- The fields expressions to map original fields of the DataSet to the fields of
the View.<T> DataSet<T> toDataSet(Table table, TypeInformation<T> typeInfo)
Table
into a DataSet
of a specified type.
The fields of the Table
are mapped to DataSet
fields as follows:
T
- The type of the resulting DataSet
.table
- The Table
to convert.typeInfo
- The TypeInformation
that specifies the type of the resulting DataSet
.DataSet
.@Deprecated BatchTableDescriptor connect(ConnectorDescriptor connectorDescriptor)
CREATE TABLE
DDL is richer than this part of the API. This method
might be refactored in the next versions. Please use executeSql(ddl)
to register a table instead.Descriptors allow for declaring the communication to external systems in an implementation-agnostic way. The classpath is scanned for suitable table factories that match the desired configuration.
The following example shows how to read from a connector using a JSON format and registering a temporary table as "MyTable":
tableEnv
.connect(
new ExternalSystemXYZ()
.version("0.11"))
.withFormat(
new Json()
.jsonSchema("{...}")
.failOnMissingField(false))
.withSchema(
new Schema()
.field("user-name", "VARCHAR").from("u_name")
.field("count", "DECIMAL")
.createTemporaryTable("MyTable")
connect
in interface TableEnvironment
connectorDescriptor
- connector descriptor describing the external systemstatic BatchTableEnvironment create(ExecutionEnvironment executionEnvironment)
TableEnvironment
for a Java batch ExecutionEnvironment
that works
with DataSet
s.
A TableEnvironment can be used to:
DataSet
to a Table
DataSet
in the TableEnvironment
's catalog
Table
in the TableEnvironment
's catalog
Table
Table
Table
into a DataSet
Table
executionEnvironment
- The Java batch ExecutionEnvironment
of the
TableEnvironment.static BatchTableEnvironment create(ExecutionEnvironment executionEnvironment, TableConfig tableConfig)
TableEnvironment
for a Java batch ExecutionEnvironment
that works
with DataSet
s.
A TableEnvironment can be used to:
DataSet
to a Table
DataSet
in the TableEnvironment
's catalog
Table
in the TableEnvironment
's catalog
Table
Table
Table
into a DataSet
Table
executionEnvironment
- The Java batch ExecutionEnvironment
of the
TableEnvironment.tableConfig
- The configuration of the TableEnvironment.Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.