Interface TableEnvironment
-
- All Known Subinterfaces:
TableEnvironmentInternal
- All Known Implementing Classes:
TableEnvironmentImpl
@PublicEvolving public interface TableEnvironment
A table environment is the base class, entry point, and central context for creating Table and SQL API programs.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:
- Connecting to external systems.
- Registering and retrieving
Table
s and other meta objects from a catalog. - Executing SQL statements.
- Offering further configuration options.
The syntax for path in methods such as
createTemporaryView(String, Table)
is following[[catalog-name.]database-name.]object-name
, where the catalog name and database are optional. For path resolution seeuseCatalog(String)
anduseDatabase(String)
.Example:
`cat.1`.`db`.`Table`
resolves to an object named 'Table' in a catalog named 'cat.1' and database named 'db'.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.
-
-
Method Summary
All Methods Static Methods Instance Methods Abstract Methods Default Methods Deprecated Methods Modifier and Type Method Description CompiledPlan
compilePlanSql(String stmt)
Compiles a SQL DML statement into aCompiledPlan
.static TableEnvironment
create(org.apache.flink.configuration.Configuration configuration)
Creates a table environment that is the entry point and central context for creating Table and SQL API programs.static TableEnvironment
create(EnvironmentSettings settings)
Creates a table environment that is the entry point and central context for creating Table and SQL API programs.void
createCatalog(String catalogName, org.apache.flink.table.catalog.CatalogDescriptor catalogDescriptor)
Creates aCatalog
using the providedCatalogDescriptor
.void
createFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a catalog function in the given path.void
createFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass, boolean ignoreIfExists)
Registers aUserDefinedFunction
class as a catalog function in the given path.void
createFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a catalog function in the given path by the specific class name and user defined resource uri.void
createFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris, boolean ignoreIfExists)
Registers aUserDefinedFunction
class as a catalog function in the given path by the specific class name and user defined resource uri.StatementSet
createStatementSet()
Returns aStatementSet
that accepts pipelines defined by DML statements orTable
objects.void
createTable(String path, TableDescriptor descriptor)
Registers the givenTableDescriptor
as a catalog table.void
createTemporaryFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a temporary catalog function.void
createTemporaryFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a temporary catalog function in the given path by the specific class name and user defined resource uri.void
createTemporaryFunction(String path, org.apache.flink.table.functions.UserDefinedFunction functionInstance)
Registers aUserDefinedFunction
instance as a temporary catalog function.void
createTemporarySystemFunction(String name, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a temporary system function.void
createTemporarySystemFunction(String name, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a temporary system function by the specific class name and user defined resource uri.void
createTemporarySystemFunction(String name, org.apache.flink.table.functions.UserDefinedFunction functionInstance)
Registers aUserDefinedFunction
instance as a temporary system function.void
createTemporaryTable(String path, TableDescriptor descriptor)
Registers the givenTableDescriptor
as a temporary catalog table.void
createTemporaryView(String path, Table view)
Registers aTable
API object as a temporary view similar to SQL temporary views.boolean
dropFunction(String path)
Drops a catalog function registered in the given path.boolean
dropTemporaryFunction(String path)
Drops a temporary catalog function registered in the given path.boolean
dropTemporarySystemFunction(String name)
Drops a temporary system function registered under the given name.boolean
dropTemporaryTable(String path)
Drops a temporary table registered in the given path.boolean
dropTemporaryView(String path)
Drops a temporary view registered in the given path.default TableResult
executePlan(PlanReference planReference)
Shorthand fortEnv.loadPlan(planReference).execute()
.TableResult
executeSql(String statement)
Executes the given single statement and returns the execution result.default String
explainSql(String statement, ExplainDetail... extraDetails)
Returns the AST of the specified statement and the execution plan to compute the result of the given statement.String
explainSql(String statement, ExplainFormat format, ExplainDetail... extraDetails)
Returns the AST of the specified statement and the execution plan to compute the result of the given statement.Table
from(String path)
Reads a registered table and returns the resultingTable
.Table
from(TableDescriptor descriptor)
Returns aTable
backed by the givendescriptor
.Table
fromValues(Iterable<?> values)
Creates a Table from given collection of objects.default Table
fromValues(Object... values)
Creates a Table from given values.Table
fromValues(org.apache.flink.table.expressions.Expression... values)
Creates a Table from given values.Table
fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, Iterable<?> values)
Creates a Table from given collection of objects with a given row type.default Table
fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, Object... values)
Creates a Table from given collection of objects with a given row type.Table
fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, org.apache.flink.table.expressions.Expression... values)
Creates a Table from given collection of objects with a given row type.Optional<org.apache.flink.table.catalog.Catalog>
getCatalog(String catalogName)
Gets a registeredCatalog
by name.String[]
getCompletionHints(String statement, int position)
Deprecated.Will be removed in the next releaseTableConfig
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.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.ModuleEntry[]
listFullModules()
Gets an array of all loaded modules with use status in this environment.String[]
listFunctions()
Gets the names of all functions in this environment.String[]
listModules()
Gets an array of names of all used modules in this environment in resolution order.String[]
listTables()
Gets the names of all tables available in the current namespace (the current database of the current catalog).String[]
listTables(String catalogName, String databaseName)
Gets the names of all tables available in the given namespace (the given database of the given catalog).String[]
listTemporaryTables()
Gets the names of all temporary tables and views available in the current namespace (the current database of the current catalog).String[]
listTemporaryViews()
Gets the names of all temporary views available in the current namespace (the current database of the current catalog).String[]
listUserDefinedFunctions()
Gets the names of all user defined functions registered in this environment.String[]
listViews()
Gets the names of all views available in the current namespace (the current database of the current catalog).void
loadModule(String moduleName, org.apache.flink.table.module.Module module)
Loads aModule
under a unique name.CompiledPlan
loadPlan(PlanReference planReference)
Loads a plan from aPlanReference
into aCompiledPlan
.void
registerCatalog(String catalogName, org.apache.flink.table.catalog.Catalog catalog)
Deprecated.UsecreateCatalog(String, CatalogDescriptor)
instead.void
registerFunction(String name, org.apache.flink.table.functions.ScalarFunction function)
Deprecated.void
registerTable(String name, Table table)
Deprecated.Table
scan(String... tablePath)
Deprecated.usefrom(String)
Table
sqlQuery(String query)
Evaluates a SQL query on registered tables and returns aTable
object describing the pipeline for further transformations.void
unloadModule(String moduleName)
Unloads aModule
with given name.void
useCatalog(String catalogName)
Sets the current catalog to the given value.void
useDatabase(String databaseName)
Sets the current default database.void
useModules(String... moduleNames)
Enable modules in use with declared name order.
-
-
-
Method Detail
-
create
static TableEnvironment create(EnvironmentSettings settings)
Creates a table environment that is the entry point and central context for creating Table and SQL API programs.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:
- Connecting to external systems.
- Registering and retrieving
Table
s and other meta objects from a catalog. - Executing SQL statements.
- Offering further configuration options.
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.
- Parameters:
settings
- The environment settings used to instantiate theTableEnvironment
.
-
create
static TableEnvironment create(org.apache.flink.configuration.Configuration configuration)
Creates a table environment that is the entry point and central context for creating Table and SQL API programs.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:
- Connecting to external systems.
- Registering and retrieving
Table
s and other meta objects from a catalog. - Executing SQL statements.
- Offering further configuration options.
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.
- Parameters:
configuration
- The specified options are used to instantiate theTableEnvironment
.
-
fromValues
default Table fromValues(Object... values)
Creates a Table from given values.Examples:
You can use a
row(...)
expression to create a composite rows:tEnv.fromValues( row(1, "ABC"), row(2L, "ABCDE") )
will produce a Table with a schema as follows:
root |-- f0: BIGINT NOT NULL // original types INT and BIGINT are generalized to BIGINT |-- f1: VARCHAR(5) NOT NULL // original types CHAR(3) and CHAR(5) are generalized to VARCHAR(5) // it uses VARCHAR instead of CHAR so that no padding is applied
The method will derive the types automatically from the input expressions. If types at a certain position differ, the method will try to find a common super type for all types. If a common super type does not exist, an exception will be thrown. If you want to specify the requested type explicitly see
fromValues(AbstractDataType, Object...)
.It is also possible to use
Row
object instead ofrow
expressions.ROWs that are a result of e.g. a function call are not flattened
public class RowFunction extends ScalarFunction { {@literal @}DataTypeHint("ROW<f0 BIGINT, f1 VARCHAR(5)>") Row eval(); } tEnv.fromValues( call(new RowFunction()), call(new RowFunction()) )
will produce a Table with a schema as follows:
root |-- f0: ROW<`f0` BIGINT, `f1` VARCHAR(5)>
The row constructor can be dropped to create a table with a single column:
ROWs that are a result of e.g. a function call are not flattened
tEnv.fromValues( 1, 2L, 3 )
will produce a Table with a schema as follows:
root |-- f0: BIGINT NOT NULL
- Parameters:
values
- Expressions for constructing rows of the VALUES table.
-
fromValues
default Table fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, Object... values)
Creates a Table from given collection of objects with a given row type.The difference between this method and
fromValues(Object...)
is that the schema can be manually adjusted. It might be helpful for assigning more generic types like e.g. DECIMAL or naming the columns.Examples:
tEnv.fromValues( DataTypes.ROW( DataTypes.FIELD("id", DataTypes.DECIMAL(10, 2)), DataTypes.FIELD("name", DataTypes.STRING()) ), row(1, "ABC"), row(2L, "ABCDE") )
will produce a Table with a schema as follows:
root |-- id: DECIMAL(10, 2) |-- name: STRING
For more examples see
fromValues(Object...)
.- Parameters:
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.- See Also:
fromValues(Object...)
-
fromValues
Table fromValues(org.apache.flink.table.expressions.Expression... values)
Creates a Table from given values.Examples:
You can use a
row(...)
expression to create a composite rows:tEnv.fromValues( row(1, "ABC"), row(2L, "ABCDE") )
will produce a Table with a schema as follows:
root |-- f0: BIGINT NOT NULL // original types INT and BIGINT are generalized to BIGINT |-- f1: VARCHAR(5) NOT NULL // original types CHAR(3) and CHAR(5) are generalized to VARCHAR(5) * // it uses VARCHAR instead of CHAR so that no padding is applied
The method will derive the types automatically from the input expressions. If types at a certain position differ, the method will try to find a common super type for all types. If a common super type does not exist, an exception will be thrown. If you want to specify the requested type explicitly see
fromValues(AbstractDataType, Expression...)
.It is also possible to use
Row
object instead ofrow
expressions.ROWs that are a result of e.g. a function call are not flattened
public class RowFunction extends ScalarFunction { {@literal @}DataTypeHint("ROW<f0 BIGINT, f1 VARCHAR(5)>") Row eval(); } tEnv.fromValues( call(new RowFunction()), call(new RowFunction()) )
will produce a Table with a schema as follows:
root |-- f0: ROW<`f0` BIGINT, `f1` VARCHAR(5)>
The row constructor can be dropped to create a table with a single column:
ROWs that are a result of e.g. a function call are not flattened
tEnv.fromValues( lit(1).plus(2), lit(2L), lit(3) )
will produce a Table with a schema as follows:
root |-- f0: BIGINT NOT NULL
- Parameters:
values
- Expressions for constructing rows of the VALUES table.
-
fromValues
Table fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, org.apache.flink.table.expressions.Expression... values)
Creates a Table from given collection of objects with a given row type.The difference between this method and
fromValues(Expression...)
is that the schema can be manually adjusted. It might be helpful for assigning more generic types like e.g. DECIMAL or naming the columns.Examples:
tEnv.fromValues( DataTypes.ROW( DataTypes.FIELD("id", DataTypes.DECIMAL(10, 2)), DataTypes.FIELD("name", DataTypes.STRING()) ), row(1, "ABC"), row(2L, "ABCDE") )
will produce a Table with a schema as follows:
root |-- id: DECIMAL(10, 2) |-- name: STRING
For more examples see
fromValues(Expression...)
.- Parameters:
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.- See Also:
fromValues(Expression...)
-
fromValues
Table fromValues(Iterable<?> values)
Creates a Table from given collection of objects.See
fromValues(Object...)
for more explanation.- Parameters:
values
- Expressions for constructing rows of the VALUES table.- See Also:
fromValues(Object...)
-
fromValues
Table fromValues(org.apache.flink.table.types.AbstractDataType<?> rowType, Iterable<?> values)
Creates a Table from given collection of objects with a given row type.See
fromValues(AbstractDataType, Object...)
for more explanation.- Parameters:
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.- See Also:
fromValues(AbstractDataType, Object...)
-
registerCatalog
@Deprecated void registerCatalog(String catalogName, org.apache.flink.table.catalog.Catalog catalog)
Deprecated.UsecreateCatalog(String, CatalogDescriptor)
instead. The new method uses aCatalogDescriptor
to initialize the catalog instance and store theCatalogDescriptor
to theCatalogStore
.Registers aCatalog
under a unique name. All tables registered in theCatalog
can be accessed.- Parameters:
catalogName
- The name under which the catalog will be registered.catalog
- The catalog to register.
-
createCatalog
void createCatalog(String catalogName, org.apache.flink.table.catalog.CatalogDescriptor catalogDescriptor)
Creates aCatalog
using the providedCatalogDescriptor
. All table registered in theCatalog
can be accessed. TheCatalogDescriptor
will be persisted into theCatalogStore
.- Parameters:
catalogName
- The name under which the catalog will be createdcatalogDescriptor
- The catalog descriptor for creating catalog
-
getCatalog
Optional<org.apache.flink.table.catalog.Catalog> getCatalog(String catalogName)
Gets a registeredCatalog
by name.- Parameters:
catalogName
- The name to look up theCatalog
.- Returns:
- The requested catalog, empty if there is no registered catalog with given name.
-
loadModule
void loadModule(String moduleName, org.apache.flink.table.module.Module module)
Loads aModule
under a unique name. Modules will be kept in the loaded order. ValidationException is thrown when there is already a module with the same name.- Parameters:
moduleName
- name of theModule
module
- the module instance
-
useModules
void useModules(String... moduleNames)
Enable modules in use with declared name order. Modules that have been loaded but not exist in names varargs will become unused.- Parameters:
moduleNames
- module names to be used
-
unloadModule
void unloadModule(String moduleName)
Unloads aModule
with given name. ValidationException is thrown when there is no module with the given name.- Parameters:
moduleName
- name of theModule
-
registerFunction
@Deprecated void registerFunction(String name, org.apache.flink.table.functions.ScalarFunction function)
Deprecated.UsecreateTemporarySystemFunction(String, UserDefinedFunction)
instead. Please note that the new method also uses the new type system and reflective extraction logic. It might be necessary to update the function implementation as well. See the documentation ofScalarFunction
for more information on the new function design.Registers aScalarFunction
under a unique name. Replaces already existing user-defined functions under this name.
-
createTemporarySystemFunction
void createTemporarySystemFunction(String name, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a temporary system function.Compared to
createTemporaryFunction(String, Class)
, system functions are identified by a global name that is independent of the current catalog and current database. Thus, this method allows to extend the set of built-in system functions likeTRIM
,ABS
, etc.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary system function.
- Parameters:
name
- The name under which the function will be registered globally.functionClass
- The function class containing the implementation.
-
createTemporarySystemFunction
void createTemporarySystemFunction(String name, org.apache.flink.table.functions.UserDefinedFunction functionInstance)
Registers aUserDefinedFunction
instance as a temporary system function.Compared to
createTemporarySystemFunction(String, Class)
, this method takes a function instance that might have been parameterized before (e.g. through its constructor). This might be useful for more interactive sessions. Make sure that the instance isSerializable
.Compared to
createTemporaryFunction(String, UserDefinedFunction)
, system functions are identified by a global name that is independent of the current catalog and current database. Thus, this method allows to extend the set of built-in system functions likeTRIM
,ABS
, etc.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary system function.
- Parameters:
name
- The name under which the function will be registered globally.functionInstance
- The (possibly pre-configured) function instance containing the implementation.
-
dropTemporarySystemFunction
boolean dropTemporarySystemFunction(String name)
Drops a temporary system function registered under the given name.If a permanent function with the given name exists, it will be used from now on for any queries that reference this name.
- Parameters:
name
- The name under which the function has been registered globally.- Returns:
- true if a function existed under the given name and was removed
-
createFunction
void createFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a catalog function in the given path.Compared to system functions with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.
There must not be another function (temporary or permanent) registered under the same path.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.functionClass
- The function class containing the implementation.
-
createFunction
void createFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass, boolean ignoreIfExists)
Registers aUserDefinedFunction
class as a catalog function in the given path.Compared to system functions with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.functionClass
- The function class containing the implementation.ignoreIfExists
- If a function exists under the given path and this flag is set, no operation is executed. An exception is thrown otherwise.
-
createFunction
void createFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a catalog function in the given path by the specific class name and user defined resource uri.Compared to
createFunction(String, Class)
, this method allows registering a user defined function by only providing a full path class name and a list of resources that contain the implementation of the function along with its dependencies. Users don't need to initialize the function instance in advance. The resource file can be a local or remote JAR file.Compared to system functions with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.
There must not be another function (temporary or permanent) registered under the same path.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.className
- The class name of UDF to be registered.resourceUris
- The list of udf resource uris in local or remote.
-
createFunction
void createFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris, boolean ignoreIfExists)
Registers aUserDefinedFunction
class as a catalog function in the given path by the specific class name and user defined resource uri.Compared to
createFunction(String, Class)
, this method allows registering a user defined function by only providing a full path class name and a list of resources that contain the implementation of the function along with its dependencies. Users don't need to initialize the function instance in advance. The resource file can be a local or remote JAR file.Compared to system functions with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.
There must not be another function (temporary or permanent) registered under the same path.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.className
- The class name of UDF to be registered.resourceUris
- The list of udf resource uris in local or remote.ignoreIfExists
- If a function exists under the given path and this flag is set, no operation is executed. An exception is thrown otherwise.
-
createTemporaryFunction
void createTemporaryFunction(String path, Class<? extends org.apache.flink.table.functions.UserDefinedFunction> functionClass)
Registers aUserDefinedFunction
class as a temporary catalog function.Compared to
createTemporarySystemFunction(String, Class)
with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary function.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.functionClass
- The function class containing the implementation.
-
createTemporaryFunction
void createTemporaryFunction(String path, org.apache.flink.table.functions.UserDefinedFunction functionInstance)
Registers aUserDefinedFunction
instance as a temporary catalog function.Compared to
createTemporaryFunction(String, Class)
, this method takes a function instance that might have been parameterized before (e.g. through its constructor). This might be useful for more interactive sessions. Make sure that the instance isSerializable
.Compared to
createTemporarySystemFunction(String, UserDefinedFunction)
with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary function.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.functionInstance
- The (possibly pre-configured) function instance containing the implementation.
-
createTemporaryFunction
void createTemporaryFunction(String path, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a temporary catalog function in the given path by the specific class name and user defined resource uri.Compared to
createTemporaryFunction(String, Class)
, this method allows registering a user defined function by only providing a full path class name and a list of resources that contain the implementation of the function along with its dependencies. Users don't need to initialize the function instance in advance. The resource file can be a local or remote JAR file.Compared to
createTemporarySystemFunction(String, String, List)
with a globally defined name, catalog functions are always (implicitly or explicitly) identified by a catalog and database.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary function.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.className
- The class name of UDF to be registered.resourceUris
- The list udf resource uri in local or remote.
-
createTemporarySystemFunction
void createTemporarySystemFunction(String name, String className, List<org.apache.flink.table.resource.ResourceUri> resourceUris)
Registers aUserDefinedFunction
class as a temporary system function by the specific class name and user defined resource uri.Compared to
createTemporaryFunction(String, Class)
, this method allows registering a user defined function by only providing a full path class name and a list of resources that contain the implementation of the function along with its dependencies. Users don't need to initialize the function instance in advance. The resource file can be a local or remote JAR file.Temporary functions can shadow permanent ones. If a permanent function under a given name exists, it will be inaccessible in the current session. To make the permanent function available again one can drop the corresponding temporary system function.
- Parameters:
name
- The name under which the function will be registered globally.className
- The class name of UDF to be registered.resourceUris
- The list of udf resource uris in local or remote.
-
dropFunction
boolean dropFunction(String path)
Drops a catalog function registered in the given path.- Parameters:
path
- The path under which the function has been registered. See also theTableEnvironment
class description for the format of the path.- Returns:
- true if a function existed in the given path and was removed
-
dropTemporaryFunction
boolean dropTemporaryFunction(String path)
Drops a temporary catalog function registered in the given path.If a permanent function with the given path exists, it will be used from now on for any queries that reference this path.
- Parameters:
path
- The path under which the function will be registered. See also theTableEnvironment
class description for the format of the path.- Returns:
- true if a function existed in the given path and was removed
-
createTemporaryTable
void createTemporaryTable(String path, TableDescriptor descriptor)
Registers the givenTableDescriptor
as a temporary catalog table.The
descriptor
is converted into aCatalogTable
and stored in the catalog.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 one can drop the corresponding temporary object.
Examples:
tEnv.createTemporaryTable("MyTable", TableDescriptor.forConnector("datagen") .schema(Schema.newBuilder() .column("f0", DataTypes.STRING()) .build()) .option(DataGenOptions.ROWS_PER_SECOND, 10) .option("fields.f0.kind", "random") .build());
- Parameters:
path
- The path under which the table will be registered. See also theTableEnvironment
class description for the format of the path.descriptor
- Template for creating aCatalogTable
instance.
-
createTable
void createTable(String path, TableDescriptor descriptor)
Registers the givenTableDescriptor
as a catalog table.The
descriptor
is converted into aCatalogTable
and stored in the catalog.If the table should not be permanently stored in a catalog, use
createTemporaryTable(String, TableDescriptor)
instead.Examples:
tEnv.createTable("MyTable", TableDescriptor.forConnector("datagen") .schema(Schema.newBuilder() .column("f0", DataTypes.STRING()) .build()) .option(DataGenOptions.ROWS_PER_SECOND, 10) .option("fields.f0.kind", "random") .build());
- Parameters:
path
- The path under which the table will be registered. See also theTableEnvironment
class description for the format of the path.descriptor
- Template for creating aCatalogTable
instance.
-
registerTable
@Deprecated void registerTable(String name, Table table)
Deprecated.Registers aTable
under a unique name in the TableEnvironment's catalog. Registered tables can be referenced in SQL queries.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 one can drop the corresponding temporary object.
- Parameters:
name
- The name under which the table will be registered.table
- The table to register.
-
createTemporaryView
void createTemporaryView(String path, Table view)
Registers aTable
API object as a temporary view similar to SQL temporary views.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 one can drop the corresponding temporary object.
- Parameters:
path
- The path under which the view will be registered. See also theTableEnvironment
class description for the format of the path.view
- The view to register.
-
scan
@Deprecated Table scan(String... tablePath)
Deprecated.usefrom(String)
Scans a registered table and returns the resultingTable
.A table to scan must be registered in the
TableEnvironment
. It can be either directly registered or be an external member of aCatalog
.See the documentation of
useDatabase(String)
oruseCatalog(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");
- Parameters:
tablePath
- The path of the table to scan.- Returns:
- The resulting
Table
. - See Also:
useCatalog(String)
,useDatabase(String)
-
from
Table from(String path)
Reads a registered table and returns the resultingTable
.A table to scan must be registered in the
TableEnvironment
.See the documentation of
useDatabase(String)
oruseCatalog(String)
for the rules on the path resolution.Examples:
Reading a table from default catalog and database.
Table tab = tableEnv.from("tableName");
Reading a table from a registered catalog.
Table tab = tableEnv.from("catalogName.dbName.tableName");
Reading a table from a registered catalog with escaping. Dots in e.g. a database name must be escaped.
Table tab = tableEnv.from("catalogName.`db.Name`.Table");
Note that the returned
Table
is an API object and only contains a pipeline description. It actually corresponds to a view in SQL terms. CallExecutable.execute()
to trigger an execution.- Parameters:
path
- The path of a table API object to scan.- Returns:
- The
Table
object describing the pipeline for further transformations. - See Also:
useCatalog(String)
,useDatabase(String)
-
from
Table from(TableDescriptor descriptor)
Returns aTable
backed by the givendescriptor
.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 temporary tables. In such cases, it is recommended to register it under a name usingcreateTemporaryTable(String, TableDescriptor)
and reference it viafrom(String)
.Examples:
Table table = tEnv.from(TableDescriptor.forConnector("datagen") .schema(Schema.newBuilder() .column("f0", DataTypes.STRING()) .build()) .build());
Note that the returned
Table
is an API object and only contains a pipeline description. It actually corresponds to a view in SQL terms. CallExecutable.execute()
to trigger an execution.- Returns:
- The
Table
object describing the pipeline for further transformations.
-
listCatalogs
String[] listCatalogs()
Gets the names of all catalogs registered in this environment.- Returns:
- A list of the names of all registered catalogs.
-
listModules
String[] listModules()
Gets an array of names of all used modules in this environment in resolution order.- Returns:
- A list of the names of used modules in resolution order.
-
listFullModules
ModuleEntry[] listFullModules()
Gets an array of all loaded modules with use status in this environment. Used modules are kept in resolution order.- Returns:
- A list of name and use status entries of all loaded modules.
-
listDatabases
String[] listDatabases()
Gets the names of all databases registered in the current catalog.- Returns:
- A list of the names of all registered databases in the current catalog.
-
listTables
String[] listTables()
Gets the names of all tables available in the current namespace (the current database of the current catalog). It returns both temporary and permanent tables and views.- Returns:
- A list of the names of all registered tables in the current database of the current catalog.
- See Also:
listTemporaryTables()
,listTemporaryViews()
-
listTables
String[] listTables(String catalogName, String databaseName)
Gets the names of all tables available in the given namespace (the given database of the given catalog). It returns both temporary and permanent tables and views.- Returns:
- A list of the names of all registered tables in the given database of the given catalog.
- See Also:
listTemporaryTables()
,listTemporaryViews()
-
listViews
String[] listViews()
Gets the names of all views available in the current namespace (the current database of the current catalog). It returns both temporary and permanent views.- Returns:
- A list of the names of all registered views in the current database of the current catalog.
- See Also:
listTemporaryViews()
-
listTemporaryTables
String[] listTemporaryTables()
Gets the names of all temporary tables and views available in the current namespace (the current database of the current catalog).- Returns:
- A list of the names of all registered temporary tables and views in the current database of the current catalog.
- See Also:
listTables()
-
listTemporaryViews
String[] listTemporaryViews()
Gets the names of all temporary views available in the current namespace (the current database of the current catalog).- Returns:
- A list of the names of all registered temporary views in the current database of the current catalog.
- See Also:
listTables()
-
listUserDefinedFunctions
String[] listUserDefinedFunctions()
Gets the names of all user defined functions registered in this environment.
-
listFunctions
String[] listFunctions()
Gets the names of all functions in this environment.
-
dropTemporaryTable
boolean dropTemporaryTable(String path)
Drops a temporary table registered in the given path.If a permanent table with a given path exists, it will be used from now on for any queries that reference this path.
- Returns:
- true if a table existed in the given path and was removed
-
dropTemporaryView
boolean dropTemporaryView(String path)
Drops a temporary view registered in the given path.If a permanent table or view with a given path exists, it will be used from now on for any queries that reference this path.
- Returns:
- true if a view existed in the given path and was removed
-
explainSql
default String explainSql(String statement, ExplainDetail... extraDetails)
Returns the AST of the specified statement and the execution plan to compute the result of the given statement.- Parameters:
statement
- The statement for which the AST and execution plan will be returned.extraDetails
- The extra explain details which the explain result should include, e.g. estimated cost, changelog mode for streaming, displaying execution plan in json format- Returns:
- AST and the execution plan.
-
explainSql
String explainSql(String statement, ExplainFormat format, ExplainDetail... extraDetails)
Returns the AST of the specified statement and the execution plan to compute the result of the given statement.- Parameters:
statement
- The statement for which the AST and execution plan will be returned.format
- The output format of explained plan.extraDetails
- The extra explain details which the explain result should include, e.g. estimated cost, changelog mode for streaming, displaying execution plan in json format- Returns:
- AST and the execution plan.
-
getCompletionHints
@Deprecated String[] getCompletionHints(String statement, int position)
Deprecated.Will be removed in the next releaseReturns completion hints for the given statement at the given cursor position. The completion happens case insensitively.- Parameters:
statement
- Partial or slightly incorrect SQL statementposition
- cursor position- Returns:
- completion hints that fit at the current cursor position
-
sqlQuery
Table sqlQuery(String query)
Evaluates a SQL query on registered tables and returns aTable
object describing the pipeline for further transformations.All tables and other objects referenced by the query must be registered in the
TableEnvironment
. For example, usecreateTemporaryView(String, Table)
) for referencing aTable
object orcreateTemporarySystemFunction(String, Class)
for functions.Alternatively, a
Table
object is automatically registered when itsTable#toString()
method is called, for example when it is embedded into a string. Hence, SQL queries can directly reference aTable
object inline (i.e. anonymous) as follows:Table table = ...; String tableName = table.toString(); // the table is not registered to the table environment tEnv.sqlQuery("SELECT * FROM " + tableName + " WHERE a > 12");
Note that the returned
Table
is an API object and only contains a pipeline description. It actually corresponds to a view in SQL terms. CallExecutable.execute()
to trigger an execution or useexecuteSql(String)
directly.- Parameters:
query
- The SQL query to evaluate.- Returns:
- The
Table
object describing the pipeline for further transformations.
-
executeSql
TableResult executeSql(String statement)
Executes the given single statement and returns the execution result.The statement can be DDL/DML/DQL/SHOW/DESCRIBE/EXPLAIN/USE. For DML and DQL, this method returns
TableResult
once the job has been submitted. For DDL and DCL statements,TableResult
is returned once the operation has finished.If multiple pipelines should insert data into one or more sink tables as part of a single execution, use a
StatementSet
(seecreateStatementSet()
).By default, all DML operations are executed asynchronously. Use
TableResult.await()
orTableResult.getJobClient()
to monitor the execution. SetTableConfigOptions.TABLE_DML_SYNC
for always synchronous execution.- Returns:
- content for DQL/SHOW/DESCRIBE/EXPLAIN, the affected row count for `DML` (-1 means unknown), or a string message ("OK") for other statements.
-
getCurrentCatalog
String getCurrentCatalog()
Gets the current default catalog name of the current session.- Returns:
- The current default catalog name that is used for the path resolution.
- See Also:
useCatalog(String)
-
useCatalog
void useCatalog(@Nullable String catalogName)
Sets the current catalog to the given value. It also sets the default database to the catalog's default one. See alsouseDatabase(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 todefault_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 You can unset the current catalog by passing a null value. If the current catalog is unset, you need to use fully qualified identifiers.
- Parameters:
catalogName
- The name of the catalog to set as the current default catalog.- See Also:
useDatabase(String)
-
getCurrentDatabase
String getCurrentDatabase()
Gets the current default database name of the running session.- Returns:
- The name of the current database of the current catalog.
- See Also:
useDatabase(String)
-
useDatabase
void useDatabase(@Nullable String databaseName)
Sets the current default database. It has to exist in the current catalog. That path will be used as the default one when looking for unqualified object names.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 todefault_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 You can unset the current database by passing a null value. If the current database is unset, you need to qualify identifiers at least with the database name.
- Parameters:
databaseName
- The name of the database to set as the current database.- See Also:
useCatalog(String)
-
getConfig
TableConfig getConfig()
Returns the table config that defines the runtime behavior of the Table API.
-
createStatementSet
StatementSet createStatementSet()
Returns aStatementSet
that accepts pipelines defined by DML statements orTable
objects. The planner can optimize all added statements together and then submit them as one job.
-
loadPlan
@Experimental CompiledPlan loadPlan(PlanReference planReference) throws org.apache.flink.table.api.TableException
Loads a plan from aPlanReference
into aCompiledPlan
.Compiled plans can be persisted and reloaded across Flink versions. They describe static pipelines to ensure backwards compatibility and enable stateful streaming job upgrades. See
CompiledPlan
and the website documentation for more information.This method will parse the input reference and will validate the plan. The returned instance can be executed via
Executable.execute()
.Note: The compiled plan feature is not supported in batch mode.
- Throws:
org.apache.flink.table.api.TableException
- if the plan cannot be loaded from the filesystem, or from classpath resources, or if the plan is invalid.
-
compilePlanSql
@Experimental CompiledPlan compilePlanSql(String stmt) throws org.apache.flink.table.api.TableException
Compiles a SQL DML statement into aCompiledPlan
.Compiled plans can be persisted and reloaded across Flink versions. They describe static pipelines to ensure backwards compatibility and enable stateful streaming job upgrades. See
CompiledPlan
and the website documentation for more information.Note: Only
INSERT INTO
is supported at the moment.Note: The compiled plan feature is not supported in batch mode.
- Throws:
org.apache.flink.table.api.TableException
- if the SQL statement is invalid or if the plan cannot be persisted.- See Also:
Executable.execute()
,loadPlan(PlanReference)
-
executePlan
@Experimental default TableResult executePlan(PlanReference planReference) throws org.apache.flink.table.api.TableException
Shorthand fortEnv.loadPlan(planReference).execute()
.- Throws:
org.apache.flink.table.api.TableException
- See Also:
loadPlan(PlanReference)
,Executable.execute()
-
-