@PublicEvolving public interface TableEnvironment
It is unified both on a language level for all JVM-based languages (i.e. there is no distinction between Scala and Java API) and for bounded and unbounded data processing.
A table environment is responsible for:
Table
s and other meta objects from a catalog.Note: This environment is meant for pure table programs. If you would like to convert from or to other Flink APIs, it might be necessary to use one of the available language-specific table environments in the corresponding bridging modules.
Modifier and Type | Method and Description |
---|---|
ConnectTableDescriptor |
connect(ConnectorDescriptor connectorDescriptor)
Creates a table source and/or table sink from a descriptor.
|
static TableEnvironment |
create(EnvironmentSettings settings)
Creates a table environment that is the entry point and central context for creating Table & SQL
API programs.
|
JobExecutionResult |
execute(String jobName)
Triggers the program execution.
|
String |
explain(boolean extended)
Returns the AST of the specified Table API and SQL queries and the execution plan to compute
the result of multiple-sinks plan.
|
String |
explain(Table table)
Returns the AST of the specified Table API and SQL queries and the execution plan to compute
the result of the given
Table . |
String |
explain(Table table,
boolean extended)
Returns the AST of the specified Table API and SQL queries and the execution plan to compute
the result of the given
Table . |
Table |
fromTableSource(TableSource<?> source)
Creates a table from a table source.
|
Optional<Catalog> |
getCatalog(String catalogName)
Gets a registered
Catalog by name. |
String[] |
getCompletionHints(String statement,
int position)
Deprecated.
Will be removed in the next release
|
TableConfig |
getConfig()
Returns the table config that defines the runtime behavior of the Table API.
|
String |
getCurrentCatalog()
Gets the current default catalog name of the current session.
|
String |
getCurrentDatabase()
Gets the current default database name of the running session.
|
ExternalCatalog |
getRegisteredExternalCatalog(String name)
Deprecated.
the
ExternalCatalog API is deprecated. Use the corresponding Catalog API. |
void |
insertInto(Table table,
String sinkPath,
String... sinkPathContinued)
|
String[] |
listCatalogs()
Gets the names of all catalogs registered in this environment.
|
String[] |
listDatabases()
Gets the names of all databases registered in the current catalog.
|
String[] |
listTables()
Gets the names of all tables registered in the current database of the current catalog.
|
String[] |
listUserDefinedFunctions()
Gets the names of all user defined functions registered in this environment.
|
void |
registerCatalog(String catalogName,
Catalog catalog)
Registers a
Catalog under a unique name. |
void |
registerExternalCatalog(String name,
ExternalCatalog externalCatalog)
Deprecated.
the
ExternalCatalog API is deprecated. Use the corresponding Catalog API. |
void |
registerFunction(String name,
ScalarFunction function)
Registers a
ScalarFunction under a unique name. |
void |
registerTable(String name,
Table table)
Registers a
Table under a unique name in the TableEnvironment's catalog. |
void |
registerTableSink(String name,
String[] fieldNames,
TypeInformation<?>[] fieldTypes,
TableSink<?> tableSink)
Deprecated.
Use
registerTableSink(String, TableSink) instead. |
void |
registerTableSink(String name,
TableSink<?> configuredSink)
Registers an external
TableSink with already configured field names and field types in
this TableEnvironment 's catalog. |
void |
registerTableSource(String name,
TableSource<?> tableSource)
Registers an external
TableSource in this TableEnvironment 's catalog. |
Table |
scan(String... tablePath)
Scans a registered table and returns the resulting
Table . |
Table |
sqlQuery(String query)
Evaluates a SQL query on registered tables and retrieves the result as a
Table . |
void |
sqlUpdate(String stmt)
Evaluates a SQL statement such as INSERT, UPDATE or DELETE; or a DDL statement;
NOTE: Currently only SQL INSERT statements and CREATE TABLE statements are supported.
|
void |
useCatalog(String catalogName)
Sets the current catalog to the given value.
|
void |
useDatabase(String databaseName)
Sets the current default database.
|
static TableEnvironment create(EnvironmentSettings settings)
It is unified both on a language level for all JVM-based languages (i.e. there is no distinction between Scala and Java API) and for bounded and unbounded data processing.
A table environment is responsible for:
Table
s and other meta objects from a catalog.Note: This environment is meant for pure table programs. If you would like to convert from or to other Flink APIs, it might be necessary to use one of the available language-specific table environments in the corresponding bridging modules.
settings
- The environment settings used to instantiate the TableEnvironment
.Table fromTableSource(TableSource<?> source)
source
- table source used as table@Deprecated void registerExternalCatalog(String name, ExternalCatalog externalCatalog)
ExternalCatalog
API is deprecated. Use the corresponding Catalog
API.ExternalCatalog
under a unique name in the TableEnvironment's schema.
All tables registered in the ExternalCatalog
can be accessed.name
- The name under which the externalCatalog will be registered.externalCatalog
- The externalCatalog to register.getCatalog(String)
,
registerCatalog(String, Catalog)
@Deprecated ExternalCatalog getRegisteredExternalCatalog(String name)
ExternalCatalog
API is deprecated. Use the corresponding Catalog
API.ExternalCatalog
by name.name
- The name to look up the ExternalCatalog
.ExternalCatalog
.getCatalog(String)
,
registerCatalog(String, Catalog)
void registerCatalog(String catalogName, Catalog catalog)
catalogName
- The name under which the catalog will be registered.catalog
- The catalog to register.Optional<Catalog> getCatalog(String catalogName)
Catalog
by name.catalogName
- The name to look up the Catalog
.void registerFunction(String name, ScalarFunction function)
ScalarFunction
under a unique name. Replaces already existing
user-defined functions under this name.void registerTable(String name, Table table)
Table
under a unique name in the TableEnvironment's catalog.
Registered tables can be referenced in SQL queries.name
- The name under which the table will be registered.table
- The table to register.void registerTableSource(String name, TableSource<?> tableSource)
TableSource
in this TableEnvironment
's catalog.
Registered tables can be referenced in SQL queries.name
- The name under which the TableSource
is registered.tableSource
- The TableSource
to register.@Deprecated void registerTableSink(String name, String[] fieldNames, TypeInformation<?>[] fieldTypes, TableSink<?> tableSink)
registerTableSink(String, TableSink)
instead.TableSink
with given field names and types in this
TableEnvironment
's catalog.
Registered sink tables can be referenced in SQL DML statements.void registerTableSink(String name, TableSink<?> configuredSink)
TableSink
with already configured field names and field types in
this TableEnvironment
's catalog.
Registered sink tables can be referenced in SQL DML statements.Table scan(String... tablePath)
Table
.
A table to scan must be registered in the TableEnvironment
. It can be either directly
registered or be an external member of a Catalog
.
See the documentation of useDatabase(String)
or
useCatalog(String)
for the rules on the path resolution.
Examples:
Scanning a directly registered table.
Table tab = tableEnv.scan("tableName");
Scanning a table from a registered catalog.
Table tab = tableEnv.scan("catalogName", "dbName", "tableName");
tablePath
- The path of the table to scan.Table
.useCatalog(String)
,
useDatabase(String)
void insertInto(Table table, String sinkPath, String... sinkPathContinued)
Table
to a TableSink
that was registered under the specified name.
See the documentation of useDatabase(String)
or
useCatalog(String)
for the rules on the path resolution.
table
- The Table to write to the sink.sinkPath
- The first part of the path of the registered TableSink
to which the Table
is
written. This is to ensure at least the name of the TableSink
is provided.sinkPathContinued
- The remaining part of the path of the registered TableSink
to which the
Table
is written.ConnectTableDescriptor connect(ConnectorDescriptor connectorDescriptor)
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 register a table source 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")
.registerSource("MyTable");
connectorDescriptor
- connector descriptor describing the external systemString[] listCatalogs()
String[] listDatabases()
String[] listTables()
String[] listUserDefinedFunctions()
String explain(Table table)
Table
.table
- The table for which the AST and execution plan will be returned.String explain(Table table, boolean extended)
Table
.table
- The table for which the AST and execution plan will be returned.extended
- if the plan should contain additional properties such as
e.g. estimated cost, traitsString explain(boolean extended)
extended
- if the plan should contain additional properties such as
e.g. estimated cost, traits@Deprecated String[] getCompletionHints(String statement, int position)
statement
- Partial or slightly incorrect SQL statementposition
- cursor positionTable sqlQuery(String query)
Table
.
All tables referenced by the query must be registered in the TableEnvironment.
A Table
is automatically registered when its Table#toString()
method is
called, for example when it is embedded into a String.
Hence, SQL queries can directly reference a Table
as follows:
Table table = ...;
String tableName = table.toString();
// the table is not registered to the table environment
tEnv.sqlQuery("SELECT * FROM tableName");
query
- The SQL query to evaluate.void sqlUpdate(String stmt)
All tables referenced by the query must be registered in the TableEnvironment.
A Table
is automatically registered when its Table#toString()
method is
called, for example when it is embedded into a String.
Hence, SQL queries can directly reference a Table
as follows:
// register the configured table sink into which the result is inserted.
tEnv.registerTableSink("sinkTable", configuredSink);
Table sourceTable = ...
String tableName = sourceTable.toString();
// sourceTable is not registered to the table environment
tEnv.sqlUpdate(s"INSERT INTO sinkTable SELECT * FROM tableName");
A DDL statement can also be executed to create a table: For example, the below DDL statement would create a CSV table named `tbl1` into the current catalog:
create table tbl1( a int, b bigint, c varchar ) with ( 'connector.type' = 'filesystem', 'format.type' = 'csv', 'connector.path' = 'xxx' )
SQL queries can directly execute as follows:
This code snippet creates a job to read data from Kafka source into a CSV sink.String sinkDDL = "create table sinkTable( a int, b varchar ) with ( 'connector.type' = 'filesystem', 'format.type' = 'csv', 'connector.path' = 'xxx' )"; String sourceDDL ="create table sourceTable( a int, b varchar ) with ( 'connector.type' = 'kafka', 'update-mode' = 'append', 'connector.topic' = 'xxx', 'connector.properties.0.key' = 'k0', 'connector.properties.0.value' = 'v0', ... )"; String query = "INSERT INTO sinkTable SELECT * FROM sourceTable"; tEnv.sqlUpdate(sourceDDL); tEnv.sqlUpdate(sinkDDL); tEnv.sqlUpdate(query); tEnv.execute("MyJob");
stmt
- The SQL statement to evaluate.String getCurrentCatalog()
useCatalog(String)
@Experimental void useCatalog(String catalogName)
useDatabase(String)
.
This is used during the resolution of object paths. Both the catalog and database are optional when referencing catalog objects such as tables, views etc. The algorithm looks for requested objects in following paths in that order:
[current-catalog].[current-database].[requested-path]
[current-catalog].[requested-path]
[requested-path]
Example:
Given structure with default catalog set to default_catalog
and default database set to
default_database
.
root: |- default_catalog |- default_database |- tab1 |- db1 |- tab1 |- cat1 |- db1 |- tab1
The following table describes resolved paths:
Requested path | Resolved path |
---|---|
tab1 | default_catalog.default_database.tab1 |
db1.tab1 | default_catalog.db1.tab1 |
cat1.db1.tab1 | cat1.db1.tab1 |
catalogName
- The name of the catalog to set as the current default catalog.useDatabase(String)
String getCurrentDatabase()
useDatabase(String)
@Experimental void useDatabase(String databaseName)
This is used during the resolution of object paths. Both the catalog and database are optional when referencing catalog objects such as tables, views etc. The algorithm looks for requested objects in following paths in that order:
[current-catalog].[current-database].[requested-path]
[current-catalog].[requested-path]
[requested-path]
Example:
Given structure with default catalog set to default_catalog
and default database set to
default_database
.
root: |- default_catalog |- default_database |- tab1 |- db1 |- tab1 |- cat1 |- db1 |- tab1The following table describes resolved paths:
Requested path | Resolved path |
---|---|
tab1 | default_catalog.default_database.tab1 |
db1.tab1 | default_catalog.db1.tab1 |
cat1.db1.tab1 | cat1.db1.tab1 |
databaseName
- The name of the database to set as the current database.useCatalog(String)
TableConfig getConfig()
JobExecutionResult execute(String jobName) throws Exception
The program execution will be logged and displayed with the provided name
NOTE:It is highly advised to set all parameters in the TableConfig
on the very beginning of the program. It is undefined what configurations values will
be used for the execution if queries are mixed with config changes. It depends on
the characteristic of the particular parameter. For some of them the value from the
point in time of query construction (e.g. the currentCatalog) will be used. On the
other hand some values might be evaluated according to the state from the time when
this method is called (e.g. timeZone).
jobName
- Desired name of the jobException
- which occurs during job execution.Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.