@Internal public class TableEnvironmentImpl extends Object implements TableEnvironmentInternal
TableEnvironment
that works exclusively with Table API interfaces. Only
TableSource
is supported as an input and TableSink
as an output. It also does not
bind to any particular StreamExecutionEnvironment
.Modifier and Type | Field and Description |
---|---|
protected Executor |
execEnv |
protected FunctionCatalog |
functionCatalog |
protected Planner |
planner |
protected ResourceManager |
resourceManager |
protected TableConfig |
tableConfig |
Modifier | Constructor and Description |
---|---|
protected |
TableEnvironmentImpl(CatalogManager catalogManager,
ModuleManager moduleManager,
ResourceManager resourceManager,
TableConfig tableConfig,
Executor executor,
FunctionCatalog functionCatalog,
Planner planner,
boolean isStreamingMode) |
Modifier and Type | Method and Description |
---|---|
CompiledPlan |
compilePlan(List<ModifyOperation> operations) |
CompiledPlan |
compilePlanSql(String stmt)
Compiles a SQL DML statement into a
CompiledPlan . |
static TableEnvironmentImpl |
create(Configuration configuration)
Creates a table environment that is the entry point and central context for creating Table
and SQL API programs.
|
static TableEnvironmentImpl |
create(EnvironmentSettings settings)
Creates a table environment that is the entry point and central context for creating Table
and SQL API programs.
|
void |
createFunction(String path,
Class<? extends UserDefinedFunction> functionClass)
Registers a
UserDefinedFunction class as a catalog function in the given path. |
void |
createFunction(String path,
Class<? extends UserDefinedFunction> functionClass,
boolean ignoreIfExists)
Registers a
UserDefinedFunction class as a catalog function in the given path. |
void |
createFunction(String path,
String className,
List<ResourceUri> resourceUris)
Registers a
UserDefinedFunction 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<ResourceUri> resourceUris,
boolean ignoreIfExists)
Registers a
UserDefinedFunction class as a catalog function in the given path by the
specific class name and user defined resource uri. |
StatementSet |
createStatementSet()
Returns a
StatementSet that accepts pipelines defined by DML statements or Table objects. |
TableImpl |
createTable(QueryOperation tableOperation) |
void |
createTable(String path,
TableDescriptor descriptor)
Registers the given
TableDescriptor as a catalog table. |
void |
createTemporaryFunction(String path,
Class<? extends UserDefinedFunction> functionClass)
Registers a
UserDefinedFunction class as a temporary catalog function. |
void |
createTemporaryFunction(String path,
String className,
List<ResourceUri> resourceUris)
Registers a
UserDefinedFunction class as a temporary catalog function in the given
path by the specific class name and user defined resource uri. |
void |
createTemporaryFunction(String path,
UserDefinedFunction functionInstance)
Registers a
UserDefinedFunction instance as a temporary catalog function. |
void |
createTemporarySystemFunction(String name,
Class<? extends UserDefinedFunction> functionClass)
Registers a
UserDefinedFunction class as a temporary system function. |
void |
createTemporarySystemFunction(String name,
String className,
List<ResourceUri> resourceUris)
Registers a
UserDefinedFunction class as a temporary system function by the specific
class name and user defined resource uri. |
void |
createTemporarySystemFunction(String name,
UserDefinedFunction functionInstance)
Registers a
UserDefinedFunction instance as a temporary system function. |
void |
createTemporaryTable(String path,
TableDescriptor descriptor)
Registers the given
TableDescriptor as a temporary catalog table. |
void |
createTemporaryView(String path,
Table view)
Registers a
Table 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.
|
TableResultInternal |
executeInternal(List<ModifyOperation> operations)
Execute the given modify operations and return the execution result.
|
TableResultInternal |
executeInternal(Operation operation)
Execute the given operation and return the execution result.
|
TableResultInternal |
executePlan(InternalPlan plan) |
TableResult |
executeSql(String statement)
Executes the given single statement and returns the execution result.
|
String |
explainInternal(List<Operation> operations,
ExplainFormat format,
ExplainDetail... extraDetails)
Returns the AST of this table and the execution plan to compute the result of this table.
|
String |
explainPlan(InternalPlan compiledPlan,
ExplainDetail... extraDetails) |
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 resulting
Table . |
Table |
from(TableDescriptor descriptor)
Returns a
Table backed by the given descriptor . |
Table |
fromTableSource(TableSource<?> source)
Creates a table from a table source.
|
Table |
fromValues(AbstractDataType<?> rowType,
Expression... values)
Creates a Table from given collection of objects with a given row type.
|
Table |
fromValues(AbstractDataType<?> rowType,
Iterable<?> values)
Creates a Table from given collection of objects with a given row type.
|
Table |
fromValues(AbstractDataType<?> rowType,
Object... values)
Creates a Table from given collection of objects with a given row type.
|
Table |
fromValues(Expression... values)
Creates a Table from given values.
|
Table |
fromValues(Iterable<?> values)
Creates a Table from given collection of objects.
|
Table |
fromValues(Object... values)
Creates a Table from given values.
|
Optional<Catalog> |
getCatalog(String catalogName)
Gets a registered
Catalog by name. |
CatalogManager |
getCatalogManager()
Returns a
CatalogManager that deals with all catalog objects. |
String[] |
getCompletionHints(String statement,
int position)
Returns completion hints for the given statement at the given cursor position.
|
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.
|
ExtendedOperationExecutor |
getExtendedOperationExecutor() |
OperationTreeBuilder |
getOperationTreeBuilder()
Returns a
OperationTreeBuilder that can create QueryOperation s. |
Parser |
getParser()
Return a
Parser that provides methods for parsing a SQL string. |
Planner |
getPlanner() |
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 catalog,
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,
Module module)
Loads a
Module under a unique name. |
CompiledPlan |
loadPlan(PlanReference planReference)
Loads a plan from a
PlanReference into a CompiledPlan . |
protected QueryOperation |
qualifyQueryOperation(ObjectIdentifier identifier,
QueryOperation queryOperation)
Subclasses can override this method to transform the given QueryOperation to a new one with
the qualified object identifier.
|
void |
registerCatalog(String catalogName,
Catalog catalog)
Registers a
Catalog under a unique name. |
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 |
registerTableSinkInternal(String name,
TableSink<?> tableSink)
Registers an external
TableSink with already configured field names and field types
in this TableEnvironment 's catalog. |
void |
registerTableSourceInternal(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 returns a
Table object describing the
pipeline for further transformations. |
protected List<Transformation<?>> |
translate(List<ModifyOperation> modifyOperations) |
void |
unloadModule(String moduleName)
Unloads a
Module 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.
|
protected void |
validateTableSource(TableSource<?> tableSource)
Subclasses can override this method to add additional checks.
|
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
explainInternal
executePlan, explainSql
protected final ResourceManager resourceManager
protected final TableConfig tableConfig
protected final Executor execEnv
protected final FunctionCatalog functionCatalog
protected final Planner planner
protected TableEnvironmentImpl(CatalogManager catalogManager, ModuleManager moduleManager, ResourceManager resourceManager, TableConfig tableConfig, Executor executor, FunctionCatalog functionCatalog, Planner planner, boolean isStreamingMode)
public static TableEnvironmentImpl create(Configuration configuration)
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.
create
in interface TableEnvironment
configuration
- The specified options are used to instantiate the TableEnvironment
.public static TableEnvironmentImpl create(EnvironmentSettings settings)
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.
create
in interface TableEnvironment
settings
- The environment settings used to instantiate the TableEnvironment
.public Table fromValues(Object... values)
TableEnvironment
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 TableEnvironment.fromValues(AbstractDataType, Object...)
.
It is also possible to use Row
object instead of row
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
fromValues
in interface TableEnvironment
values
- Expressions for constructing rows of the VALUES table.public Table fromValues(AbstractDataType<?> rowType, Object... values)
TableEnvironment
The difference between this method and TableEnvironment.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 TableEnvironment.fromValues(Object...)
.
fromValues
in interface TableEnvironment
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.TableEnvironment.fromValues(Object...)
public Table fromValues(Expression... values)
TableEnvironment
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 TableEnvironment.fromValues(AbstractDataType, Expression...)
.
It is also possible to use Row
object instead of row
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
fromValues
in interface TableEnvironment
values
- Expressions for constructing rows of the VALUES table.public Table fromValues(AbstractDataType<?> rowType, Expression... values)
TableEnvironment
The difference between this method and TableEnvironment.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 TableEnvironment.fromValues(Expression...)
.
fromValues
in interface TableEnvironment
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.TableEnvironment.fromValues(Expression...)
public Table fromValues(Iterable<?> values)
TableEnvironment
See TableEnvironment.fromValues(Object...)
for more explanation.
fromValues
in interface TableEnvironment
values
- Expressions for constructing rows of the VALUES table.TableEnvironment.fromValues(Object...)
public Table fromValues(AbstractDataType<?> rowType, Iterable<?> values)
TableEnvironment
See TableEnvironment.fromValues(AbstractDataType, Object...)
for more explanation.
fromValues
in interface TableEnvironment
rowType
- Expected row type for the values.values
- Expressions for constructing rows of the VALUES table.TableEnvironment.fromValues(AbstractDataType, Object...)
@VisibleForTesting public Planner getPlanner()
public Table fromTableSource(TableSource<?> source)
TableEnvironmentInternal
fromTableSource
in interface TableEnvironmentInternal
source
- table source used as tablepublic void registerCatalog(String catalogName, Catalog catalog)
TableEnvironment
registerCatalog
in interface TableEnvironment
catalogName
- The name under which the catalog will be registered.catalog
- The catalog to register.public Optional<Catalog> getCatalog(String catalogName)
TableEnvironment
Catalog
by name.getCatalog
in interface TableEnvironment
catalogName
- The name to look up the Catalog
.public void loadModule(String moduleName, Module module)
TableEnvironment
Module
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.loadModule
in interface TableEnvironment
moduleName
- name of the Module
module
- the module instancepublic void useModules(String... moduleNames)
TableEnvironment
useModules
in interface TableEnvironment
moduleNames
- module names to be usedpublic void unloadModule(String moduleName)
TableEnvironment
Module
with given name. ValidationException is thrown when there is no
module with the given name.unloadModule
in interface TableEnvironment
moduleName
- name of the Module
public void registerFunction(String name, ScalarFunction function)
TableEnvironment
ScalarFunction
under a unique name. Replaces already existing
user-defined functions under this name.registerFunction
in interface TableEnvironment
public void createTemporarySystemFunction(String name, Class<? extends UserDefinedFunction> functionClass)
TableEnvironment
UserDefinedFunction
class as a temporary system function.
Compared to TableEnvironment.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 like TRIM
,
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.
createTemporarySystemFunction
in interface TableEnvironment
name
- The name under which the function will be registered globally.functionClass
- The function class containing the implementation.public void createTemporarySystemFunction(String name, UserDefinedFunction functionInstance)
TableEnvironment
UserDefinedFunction
instance as a temporary system function.
Compared to TableEnvironment.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 is Serializable
.
Compared to TableEnvironment.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
like TRIM
, 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.
createTemporarySystemFunction
in interface TableEnvironment
name
- The name under which the function will be registered globally.functionInstance
- The (possibly pre-configured) function instance containing the
implementation.public void createTemporarySystemFunction(String name, String className, List<ResourceUri> resourceUris)
TableEnvironment
UserDefinedFunction
class as a temporary system function by the specific
class name and user defined resource uri.
Compared to TableEnvironment.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.
createTemporarySystemFunction
in interface TableEnvironment
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.public boolean dropTemporarySystemFunction(String name)
TableEnvironment
If a permanent function with the given name exists, it will be used from now on for any queries that reference this name.
dropTemporarySystemFunction
in interface TableEnvironment
name
- The name under which the function has been registered globally.public void createFunction(String path, Class<? extends UserDefinedFunction> functionClass)
TableEnvironment
UserDefinedFunction
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.
createFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
class description for the format of the path.functionClass
- The function class containing the implementation.public void createFunction(String path, Class<? extends UserDefinedFunction> functionClass, boolean ignoreIfExists)
TableEnvironment
UserDefinedFunction
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.
createFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
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.public void createFunction(String path, String className, List<ResourceUri> resourceUris)
TableEnvironment
UserDefinedFunction
class as a catalog function in the given path by the
specific class name and user defined resource uri.
Compared to TableEnvironment.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.
createFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
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.public void createFunction(String path, String className, List<ResourceUri> resourceUris, boolean ignoreIfExists)
TableEnvironment
UserDefinedFunction
class as a catalog function in the given path by the
specific class name and user defined resource uri.
Compared to TableEnvironment.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.
createFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
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.public boolean dropFunction(String path)
TableEnvironment
dropFunction
in interface TableEnvironment
path
- The path under which the function has been registered. See also the TableEnvironment
class description for the format of the path.public void createTemporaryFunction(String path, Class<? extends UserDefinedFunction> functionClass)
TableEnvironment
UserDefinedFunction
class as a temporary catalog function.
Compared to TableEnvironment.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.
createTemporaryFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
class description for the format of the path.functionClass
- The function class containing the implementation.public void createTemporaryFunction(String path, UserDefinedFunction functionInstance)
TableEnvironment
UserDefinedFunction
instance as a temporary catalog function.
Compared to TableEnvironment.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 is Serializable
.
Compared to TableEnvironment.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.
createTemporaryFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
class description for the format of the path.functionInstance
- The (possibly pre-configured) function instance containing the
implementation.public void createTemporaryFunction(String path, String className, List<ResourceUri> resourceUris)
TableEnvironment
UserDefinedFunction
class as a temporary catalog function in the given
path by the specific class name and user defined resource uri.
Compared to TableEnvironment.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 TableEnvironment.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.
createTemporaryFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
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.public boolean dropTemporaryFunction(String path)
TableEnvironment
If a permanent function with the given path exists, it will be used from now on for any queries that reference this path.
dropTemporaryFunction
in interface TableEnvironment
path
- The path under which the function will be registered. See also the TableEnvironment
class description for the format of the path.public void createTemporaryTable(String path, TableDescriptor descriptor)
TableEnvironment
TableDescriptor
as a temporary catalog table.
The descriptor
is converted into a CatalogTable
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());
createTemporaryTable
in interface TableEnvironment
path
- The path under which the table will be registered. See also the TableEnvironment
class description for the format of the path.descriptor
- Template for creating a CatalogTable
instance.public void createTable(String path, TableDescriptor descriptor)
TableEnvironment
TableDescriptor
as a catalog table.
The descriptor
is converted into a CatalogTable
and stored
in the catalog.
If the table should not be permanently stored in a catalog, use TableEnvironment.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());
createTable
in interface TableEnvironment
path
- The path under which the table will be registered. See also the TableEnvironment
class description for the format of the path.descriptor
- Template for creating a CatalogTable
instance.public void registerTable(String name, Table table)
TableEnvironment
Table
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.
registerTable
in interface TableEnvironment
name
- The name under which the table will be registered.table
- The table to register.public void createTemporaryView(String path, Table view)
TableEnvironment
Table
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.
createTemporaryView
in interface TableEnvironment
path
- The path under which the view will be registered. See also the TableEnvironment
class description for the format of the path.view
- The view to register.public Table scan(String... tablePath)
TableEnvironment
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 TableEnvironment.useDatabase(String)
or TableEnvironment.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");
scan
in interface TableEnvironment
tablePath
- The path of the table to scan.Table
.TableEnvironment.useCatalog(String)
,
TableEnvironment.useDatabase(String)
public Table from(String path)
TableEnvironment
Table
.
A table to scan must be registered in the TableEnvironment
.
See the documentation of TableEnvironment.useDatabase(String)
or TableEnvironment.useCatalog(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. Call Executable.execute()
to trigger an execution.
from
in interface TableEnvironment
path
- The path of a table API object to scan.Table
object describing the pipeline for further transformations.TableEnvironment.useCatalog(String)
,
TableEnvironment.useDatabase(String)
public Table from(TableDescriptor descriptor)
TableEnvironment
Table
backed by the given descriptor
.
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 using TableEnvironment.createTemporaryTable(String,
TableDescriptor)
and reference it via TableEnvironment.from(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. Call Executable.execute()
to trigger an execution.
from
in interface TableEnvironment
Table
object describing the pipeline for further transformations.public String[] listCatalogs()
TableEnvironment
listCatalogs
in interface TableEnvironment
public String[] listModules()
TableEnvironment
listModules
in interface TableEnvironment
public ModuleEntry[] listFullModules()
TableEnvironment
listFullModules
in interface TableEnvironment
public String[] listDatabases()
TableEnvironment
listDatabases
in interface TableEnvironment
public String[] listTables()
TableEnvironment
listTables
in interface TableEnvironment
TableEnvironment.listTemporaryTables()
,
TableEnvironment.listTemporaryViews()
public String[] listTables(String catalog, String databaseName)
TableEnvironment
listTables
in interface TableEnvironment
TableEnvironment.listTemporaryTables()
,
TableEnvironment.listTemporaryViews()
public String[] listViews()
TableEnvironment
listViews
in interface TableEnvironment
TableEnvironment.listTemporaryViews()
public String[] listTemporaryTables()
TableEnvironment
listTemporaryTables
in interface TableEnvironment
TableEnvironment.listTables()
public String[] listTemporaryViews()
TableEnvironment
listTemporaryViews
in interface TableEnvironment
TableEnvironment.listTables()
public boolean dropTemporaryTable(String path)
TableEnvironment
If a permanent table with a given path exists, it will be used from now on for any queries that reference this path.
dropTemporaryTable
in interface TableEnvironment
public boolean dropTemporaryView(String path)
TableEnvironment
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.
dropTemporaryView
in interface TableEnvironment
public String[] listUserDefinedFunctions()
TableEnvironment
listUserDefinedFunctions
in interface TableEnvironment
public String[] listFunctions()
TableEnvironment
listFunctions
in interface TableEnvironment
public String explainSql(String statement, ExplainFormat format, ExplainDetail... extraDetails)
TableEnvironment
explainSql
in interface TableEnvironment
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 formatpublic String explainInternal(List<Operation> operations, ExplainFormat format, ExplainDetail... extraDetails)
TableEnvironmentInternal
explainInternal
in interface TableEnvironmentInternal
operations
- The operations to be explained.format
- The output format.extraDetails
- The extra explain details which the explain result should include, e.g.
estimated cost, changelog mode for streamingpublic String[] getCompletionHints(String statement, int position)
TableEnvironment
getCompletionHints
in interface TableEnvironment
statement
- Partial or slightly incorrect SQL statementposition
- cursor positionpublic Table sqlQuery(String query)
TableEnvironment
Table
object describing the
pipeline for further transformations.
All tables and other objects referenced by the query must be registered in the TableEnvironment
. For example, use TableEnvironment.createTemporaryView(String, Table)
) for
referencing a Table
object or TableEnvironment.createTemporarySystemFunction(String, Class)
for functions.
Alternatively, a Table
object 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
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. Call Executable.execute()
to trigger an execution or use TableEnvironment.executeSql(String)
directly.
sqlQuery
in interface TableEnvironment
query
- The SQL query to evaluate.Table
object describing the pipeline for further transformations.public TableResult executeSql(String statement)
TableEnvironment
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
(see TableEnvironment.createStatementSet()
).
By default, all DML operations are executed asynchronously. Use TableResult.await()
or TableResult.getJobClient()
to monitor the execution. Set
TableConfigOptions.TABLE_DML_SYNC
for always synchronous execution.
executeSql
in interface TableEnvironment
public StatementSet createStatementSet()
TableEnvironment
StatementSet
that accepts pipelines defined by DML statements or Table
objects. The planner can optimize all added statements together and then submit them
as one job.createStatementSet
in interface TableEnvironment
public CompiledPlan loadPlan(PlanReference planReference)
TableEnvironment
PlanReference
into a CompiledPlan
.
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.
loadPlan
in interface TableEnvironment
public CompiledPlan compilePlanSql(String stmt)
TableEnvironment
CompiledPlan
.
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.
compilePlanSql
in interface TableEnvironment
Executable.execute()
,
TableEnvironment.loadPlan(PlanReference)
public TableResultInternal executePlan(InternalPlan plan)
executePlan
in interface TableEnvironmentInternal
public CompiledPlan compilePlan(List<ModifyOperation> operations)
compilePlan
in interface TableEnvironmentInternal
public TableResultInternal executeInternal(List<ModifyOperation> operations)
TableEnvironmentInternal
executeInternal
in interface TableEnvironmentInternal
operations
- The operations to be executed.public TableResultInternal executeInternal(Operation operation)
TableEnvironmentInternal
executeInternal
in interface TableEnvironmentInternal
operation
- The operation to be executed.public String getCurrentCatalog()
TableEnvironment
getCurrentCatalog
in interface TableEnvironment
TableEnvironment.useCatalog(String)
public void useCatalog(String catalogName)
TableEnvironment
TableEnvironment.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 |
useCatalog
in interface TableEnvironment
catalogName
- The name of the catalog to set as the current default catalog.TableEnvironment.useDatabase(String)
public String getCurrentDatabase()
TableEnvironment
getCurrentDatabase
in interface TableEnvironment
TableEnvironment.useDatabase(String)
public void useDatabase(String databaseName)
TableEnvironment
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 |
useDatabase
in interface TableEnvironment
databaseName
- The name of the database to set as the current database.TableEnvironment.useCatalog(String)
public TableConfig getConfig()
TableEnvironment
getConfig
in interface TableEnvironment
public Parser getParser()
TableEnvironmentInternal
Parser
that provides methods for parsing a SQL string.getParser
in interface TableEnvironmentInternal
Parser
.public ExtendedOperationExecutor getExtendedOperationExecutor()
public CatalogManager getCatalogManager()
TableEnvironmentInternal
CatalogManager
that deals with all catalog objects.getCatalogManager
in interface TableEnvironmentInternal
public OperationTreeBuilder getOperationTreeBuilder()
TableEnvironmentInternal
OperationTreeBuilder
that can create QueryOperation
s.getOperationTreeBuilder
in interface TableEnvironmentInternal
protected QueryOperation qualifyQueryOperation(ObjectIdentifier identifier, QueryOperation queryOperation)
fromDataStream(DataStream)
. But the identifier is required when converting this
QueryOperation to RelNode.protected void validateTableSource(TableSource<?> tableSource)
tableSource
- tableSource to validateprotected List<Transformation<?>> translate(List<ModifyOperation> modifyOperations)
public void registerTableSourceInternal(String name, TableSource<?> tableSource)
TableEnvironmentInternal
TableSource
in this 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.
registerTableSourceInternal
in interface TableEnvironmentInternal
name
- The name under which the TableSource
is registered.tableSource
- The TableSource
to register.public void registerTableSinkInternal(String name, TableSink<?> tableSink)
TableEnvironmentInternal
TableSink
with already configured field names and field types
in this TableEnvironment
's catalog. Registered sink tables can be referenced in SQL
DML statements.
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.
registerTableSinkInternal
in interface TableEnvironmentInternal
name
- The name under which the TableSink
is registered.tableSink
- The configured TableSink
to register.@VisibleForTesting public TableImpl createTable(QueryOperation tableOperation)
public String explainPlan(InternalPlan compiledPlan, ExplainDetail... extraDetails)
explainPlan
in interface TableEnvironmentInternal
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.