@Internal public final class StreamTableEnvironmentImpl extends AbstractStreamTableEnvironmentImpl implements StreamTableEnvironment
StreamTableEnvironment
. This enables conversions from/to
DataStream
.
It binds to a given StreamExecutionEnvironment
.
executionEnvironment
execEnv, functionCatalog, planner, resourceManager, tableConfig
Constructor and Description |
---|
StreamTableEnvironmentImpl(CatalogManager catalogManager,
ModuleManager moduleManager,
ResourceManager resourceManager,
FunctionCatalog functionCatalog,
TableConfig tableConfig,
StreamExecutionEnvironment executionEnvironment,
Planner planner,
Executor executor,
boolean isStreamingMode) |
Modifier and Type | Method and Description |
---|---|
static StreamTableEnvironment |
create(StreamExecutionEnvironment executionEnvironment,
EnvironmentSettings settings)
Creates a table environment that is the entry point and central context for creating Table
and SQL API programs that integrate with the Java-specific
DataStream API. |
StreamStatementSet |
createStatementSet()
Returns a
StatementSet that accepts pipelines defined by DML statements or Table objects. |
<T> void |
createTemporaryView(String path,
DataStream<T> dataStream)
Creates a view from the given
DataStream in a given path. |
<T> void |
createTemporaryView(String path,
DataStream<T> dataStream,
Expression... fields)
Creates a view from the given
DataStream in a given path with specified field names. |
<T> void |
createTemporaryView(String path,
DataStream<T> dataStream,
Schema schema)
Creates a view from the given
DataStream in a given path. |
Table |
fromChangelogStream(DataStream<Row> dataStream)
Converts the given
DataStream of changelog entries into a Table . |
Table |
fromChangelogStream(DataStream<Row> dataStream,
Schema schema)
Converts the given
DataStream of changelog entries into a Table . |
Table |
fromChangelogStream(DataStream<Row> dataStream,
Schema schema,
ChangelogMode changelogMode)
Converts the given
DataStream of changelog entries into a Table . |
<T> Table |
fromDataStream(DataStream<T> dataStream)
Converts the given
DataStream into a Table . |
<T> Table |
fromDataStream(DataStream<T> dataStream,
Expression... fields)
Converts the given
DataStream into a Table with specified field names. |
<T> Table |
fromDataStream(DataStream<T> dataStream,
Schema schema)
Converts the given
DataStream into a Table . |
<T> void |
registerDataStream(String name,
DataStream<T> dataStream)
Creates a view from the given
DataStream . |
<T,ACC> void |
registerFunction(String name,
AggregateFunction<T,ACC> aggregateFunction)
Registers an
AggregateFunction under a unique name in the TableEnvironment's catalog. |
<T,ACC> void |
registerFunction(String name,
TableAggregateFunction<T,ACC> tableAggregateFunction)
Registers an
TableAggregateFunction under a unique name in the TableEnvironment's
catalog. |
<T> void |
registerFunction(String name,
TableFunction<T> tableFunction)
Registers a
TableFunction under a unique name in the TableEnvironment's catalog. |
<T> DataStream<T> |
toAppendStream(Table table,
Class<T> clazz)
Converts the given
Table into an append DataStream of a specified type. |
<T> DataStream<T> |
toAppendStream(Table table,
TypeInformation<T> typeInfo)
Converts the given
Table into an append DataStream of a specified type. |
DataStream<Row> |
toChangelogStream(Table table)
Converts the given
Table into a DataStream of changelog entries. |
DataStream<Row> |
toChangelogStream(Table table,
Schema targetSchema)
Converts the given
Table into a DataStream of changelog entries. |
DataStream<Row> |
toChangelogStream(Table table,
Schema targetSchema,
ChangelogMode changelogMode)
Converts the given
Table into a DataStream of changelog entries. |
DataStream<Row> |
toDataStream(Table table)
Converts the given
Table into a DataStream . |
<T> DataStream<T> |
toDataStream(Table table,
AbstractDataType<?> targetDataType)
|
<T> DataStream<T> |
toDataStream(Table table,
Class<T> targetClass)
|
<T> DataStream<Tuple2<Boolean,T>> |
toRetractStream(Table table,
Class<T> clazz)
Converts the given
Table into a DataStream of add and retract messages. |
<T> DataStream<Tuple2<Boolean,T>> |
toRetractStream(Table table,
TypeInformation<T> typeInfo)
Converts the given
Table into a DataStream of add and retract messages. |
protected void |
validateTableSource(TableSource<?> tableSource)
Subclasses can override this method to add additional checks.
|
asQueryOperation, attachAsDataStream, execEnv, extractTypeInformation, fromStreamInternal, lookupExecutor, qualifyQueryOperation, toStreamInternal, toStreamInternal, validateTimeCharacteristic, wrapWithChangeFlag
compilePlan, compilePlanSql, create, create, createCatalog, createFunction, createFunction, createFunction, createFunction, createTable, createTable, createTemporaryFunction, createTemporaryFunction, createTemporaryFunction, createTemporarySystemFunction, createTemporarySystemFunction, createTemporarySystemFunction, createTemporaryTable, createTemporaryView, dropFunction, dropTemporaryFunction, dropTemporarySystemFunction, dropTemporaryTable, dropTemporaryView, executeCachedPlanInternal, executeInternal, executeInternal, executePlan, executeSql, explainInternal, explainPlan, explainSql, from, from, fromTableSource, fromValues, fromValues, fromValues, fromValues, fromValues, fromValues, generatePipelineFromQueryOperation, getCatalog, getCatalogManager, getCompletionHints, getConfig, getCurrentCatalog, getCurrentDatabase, getOperationTreeBuilder, getParser, getPlanner, listCatalogs, listDatabases, listFullModules, listFunctions, listModules, listTables, listTables, listTemporaryTables, listTemporaryViews, listUserDefinedFunctions, listViews, loadModule, loadPlan, registerCatalog, registerFunction, registerTable, registerTableSinkInternal, registerTableSourceInternal, scan, sqlQuery, translate, unloadModule, useCatalog, useDatabase, useModules
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
create
compilePlanSql, create, create, createCatalog, createFunction, createFunction, createFunction, createFunction, createTable, createTemporaryFunction, createTemporaryFunction, createTemporaryFunction, createTemporarySystemFunction, createTemporarySystemFunction, createTemporarySystemFunction, createTemporaryTable, createTemporaryView, dropFunction, dropTemporaryFunction, dropTemporarySystemFunction, dropTemporaryTable, dropTemporaryView, executePlan, executeSql, explainSql, explainSql, from, from, fromValues, fromValues, fromValues, fromValues, fromValues, fromValues, getCatalog, getCompletionHints, getConfig, getCurrentCatalog, getCurrentDatabase, listCatalogs, listDatabases, listFullModules, listFunctions, listModules, listTables, listTables, listTemporaryTables, listTemporaryViews, listUserDefinedFunctions, listViews, loadModule, loadPlan, registerCatalog, registerFunction, registerTable, scan, sqlQuery, unloadModule, useCatalog, useDatabase, useModules
explainInternal
public StreamTableEnvironmentImpl(CatalogManager catalogManager, ModuleManager moduleManager, ResourceManager resourceManager, FunctionCatalog functionCatalog, TableConfig tableConfig, StreamExecutionEnvironment executionEnvironment, Planner planner, Executor executor, boolean isStreamingMode)
public static StreamTableEnvironment create(StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings)
StreamTableEnvironment
DataStream
API.
It is unified for bounded and unbounded data processing.
A stream table environment is responsible for:
DataStream
into Table
and vice-versa.
Table
s and other meta objects from a catalog.
Note: If you don't intend to use the DataStream
API, TableEnvironment
is
meant for pure table programs.
create
in interface StreamTableEnvironment
executionEnvironment
- The Java StreamExecutionEnvironment
of the TableEnvironment
.settings
- The environment settings used to instantiate the TableEnvironment
.public <T> void registerFunction(String name, TableFunction<T> tableFunction)
StreamTableEnvironment
TableFunction
under a unique name in the TableEnvironment's catalog.
Registered functions can be referenced in Table API and SQL queries.registerFunction
in interface StreamTableEnvironment
T
- The type of the output row.name
- The name under which the function is registered.tableFunction
- The TableFunction to register.public <T,ACC> void registerFunction(String name, AggregateFunction<T,ACC> aggregateFunction)
StreamTableEnvironment
AggregateFunction
under a unique name in the TableEnvironment's catalog.
Registered functions can be referenced in Table API and SQL queries.registerFunction
in interface StreamTableEnvironment
T
- The type of the output value.ACC
- The type of aggregate accumulator.name
- The name under which the function is registered.aggregateFunction
- The AggregateFunction to register.public <T,ACC> void registerFunction(String name, TableAggregateFunction<T,ACC> tableAggregateFunction)
StreamTableEnvironment
TableAggregateFunction
under a unique name in the TableEnvironment's
catalog. Registered functions can only be referenced in Table API.registerFunction
in interface StreamTableEnvironment
T
- The type of the output value.ACC
- The type of aggregate accumulator.name
- The name under which the function is registered.tableAggregateFunction
- The TableAggregateFunction to register.public <T> Table fromDataStream(DataStream<T> dataStream)
StreamTableEnvironment
DataStream
into a Table
.
Column names and types of the Table
are automatically derived from the TypeInformation
of the DataStream
. If the outermost record's TypeInformation
is a CompositeType
, it will be flattened in the first level. TypeInformation
that cannot be represented as one of the listed DataTypes
will be treated as a
black-box DataTypes.RAW(Class, TypeSerializer)
type. Thus, composite nested fields
will not be accessible.
Since the DataStream API does not support changelog processing natively, this method
assumes append-only/insert-only semantics during the stream-to-table conversion. Records of
type Row
must describe RowKind.INSERT
changes.
By default, the stream record's timestamp and watermarks are not propagated to downstream
table operations unless explicitly declared via StreamTableEnvironment.fromDataStream(DataStream, Schema)
.
If the returned table is converted back to DataStream via StreamTableEnvironment.toDataStream(Table)
,
the input DataStream of this method would be returned.
fromDataStream
in interface StreamTableEnvironment
T
- The external type of the DataStream
.dataStream
- The DataStream
to be converted.Table
.StreamTableEnvironment.fromChangelogStream(DataStream)
public <T> Table fromDataStream(DataStream<T> dataStream, Schema schema)
StreamTableEnvironment
DataStream
into a Table
.
Column names and types of the Table
are automatically derived from the TypeInformation
of the DataStream
. If the outermost record's TypeInformation
is a CompositeType
, it will be flattened in the first level. TypeInformation
that cannot be represented as one of the listed DataTypes
will be treated as a
black-box DataTypes.RAW(Class, TypeSerializer)
type. Thus, composite nested fields
will not be accessible.
Since the DataStream API does not support changelog processing natively, this method
assumes append-only/insert-only semantics during the stream-to-table conversion. Records of
class Row
must describe RowKind.INSERT
changes.
By default, the stream record's timestamp and watermarks are not propagated to downstream table operations unless explicitly declared in the input schema.
This method allows to declare a Schema
for the resulting table. The declaration is
similar to a CREATE TABLE
DDL in SQL and allows to:
DataType
DataStream
watermarks
It is possible to declare a schema without physical/regular columns. In this case, those columns will be automatically derived and implicitly put at the beginning of the schema declaration.
The following examples illustrate common schema declarations and their semantics:
// given a DataStream of Tuple2 < String , BigDecimal > // === EXAMPLE 1 === // no physical columns defined, they will be derived automatically, // e.g. BigDecimal becomes DECIMAL(38, 18) Schema.newBuilder() .columnByExpression("c1", "f1 + 42") .columnByExpression("c2", "f1 - 1") .build() // equal to: CREATE TABLE (f0 STRING, f1 DECIMAL(38, 18), c1 AS f1 + 42, c2 AS f1 - 1) // === EXAMPLE 2 === // physical columns defined, input fields and columns will be mapped by name, // columns are reordered and their data type overwritten, // all columns must be defined to show up in the final table's schema Schema.newBuilder() .column("f1", "DECIMAL(10, 2)") .columnByExpression("c", "f1 - 1") .column("f0", "STRING") .build() // equal to: CREATE TABLE (f1 DECIMAL(10, 2), c AS f1 - 1, f0 STRING) // === EXAMPLE 3 === // timestamp and watermarks can be added from the DataStream API, // physical columns will be derived automatically Schema.newBuilder() .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)") // extract timestamp into a column .watermark("rowtime", "SOURCE_WATERMARK()") // declare watermarks propagation .build() // equal to: // CREATE TABLE ( // f0 STRING, // f1 DECIMAL(38, 18), // rowtime TIMESTAMP(3) METADATA, // WATERMARK FOR rowtime AS SOURCE_WATERMARK() // )
fromDataStream
in interface StreamTableEnvironment
T
- The external type of the DataStream
.dataStream
- The DataStream
to be converted.schema
- The customized schema for the final table.Table
.StreamTableEnvironment.fromChangelogStream(DataStream, Schema)
public Table fromChangelogStream(DataStream<Row> dataStream)
StreamTableEnvironment
DataStream
of changelog entries into a Table
.
Compared to StreamTableEnvironment.fromDataStream(DataStream)
, this method consumes instances of Row
and evaluates the RowKind
flag that is contained in every record during runtime.
The runtime behavior is similar to that of a DynamicTableSource
.
This method expects a changelog containing all kinds of changes (enumerated in RowKind
) as the default ChangelogMode
. Use StreamTableEnvironment.fromChangelogStream(DataStream,
Schema, ChangelogMode)
to limit the kinds of changes (e.g. for upsert mode).
Column names and types of the Table
are automatically derived from the TypeInformation
of the DataStream
. If the outermost record's TypeInformation
is a CompositeType
, it will be flattened in the first level. TypeInformation
that cannot be represented as one of the listed DataTypes
will be treated as a
black-box DataTypes.RAW(Class, TypeSerializer)
type. Thus, composite nested fields
will not be accessible.
By default, the stream record's timestamp and watermarks are not propagated to downstream
table operations unless explicitly declared via StreamTableEnvironment.fromChangelogStream(DataStream,
Schema)
.
fromChangelogStream
in interface StreamTableEnvironment
dataStream
- The changelog stream of Row
.Table
.public Table fromChangelogStream(DataStream<Row> dataStream, Schema schema)
StreamTableEnvironment
DataStream
of changelog entries into a Table
.
Compared to StreamTableEnvironment.fromDataStream(DataStream)
, this method consumes instances of Row
and evaluates the RowKind
flag that is contained in every record during runtime.
The runtime behavior is similar to that of a DynamicTableSource
.
This method expects a changelog containing all kinds of changes (enumerated in RowKind
) as the default ChangelogMode
. Use StreamTableEnvironment.fromChangelogStream(DataStream,
Schema, ChangelogMode)
to limit the kinds of changes (e.g. for upsert mode).
Column names and types of the Table
are automatically derived from the TypeInformation
of the DataStream
. If the outermost record's TypeInformation
is a CompositeType
, it will be flattened in the first level. TypeInformation
that cannot be represented as one of the listed DataTypes
will be treated as a
black-box DataTypes.RAW(Class, TypeSerializer)
type. Thus, composite nested fields
will not be accessible.
By default, the stream record's timestamp and watermarks are not propagated to downstream table operations unless explicitly declared in the input schema.
This method allows to declare a Schema
for the resulting table. The declaration is
similar to a CREATE TABLE
DDL in SQL and allows to:
DataType
DataStream
watermarks
See StreamTableEnvironment.fromDataStream(DataStream, Schema)
for more information and examples on how
to declare a Schema
.
fromChangelogStream
in interface StreamTableEnvironment
dataStream
- The changelog stream of Row
.schema
- The customized schema for the final table.Table
.public Table fromChangelogStream(DataStream<Row> dataStream, Schema schema, ChangelogMode changelogMode)
StreamTableEnvironment
DataStream
of changelog entries into a Table
.
Compared to StreamTableEnvironment.fromDataStream(DataStream)
, this method consumes instances of Row
and evaluates the RowKind
flag that is contained in every record during runtime.
The runtime behavior is similar to that of a DynamicTableSource
.
This method requires an explicitly declared ChangelogMode
. For example, use ChangelogMode.upsert()
if the stream will not contain RowKind.UPDATE_BEFORE
, or
ChangelogMode.insertOnly()
for non-updating streams.
Column names and types of the Table
are automatically derived from the TypeInformation
of the DataStream
. If the outermost record's TypeInformation
is a CompositeType
, it will be flattened in the first level. TypeInformation
that cannot be represented as one of the listed DataTypes
will be treated as a
black-box DataTypes.RAW(Class, TypeSerializer)
type. Thus, composite nested fields
will not be accessible.
By default, the stream record's timestamp and watermarks are not propagated to downstream table operations unless explicitly declared in the input schema.
This method allows to declare a Schema
for the resulting table. The declaration is
similar to a CREATE TABLE
DDL in SQL and allows to:
DataType
DataStream
watermarks
See StreamTableEnvironment.fromDataStream(DataStream, Schema)
for more information and examples of how
to declare a Schema
.
fromChangelogStream
in interface StreamTableEnvironment
dataStream
- The changelog stream of Row
.schema
- The customized schema for the final table.changelogMode
- The expected kinds of changes in the incoming changelog.Table
.public <T> void createTemporaryView(String path, DataStream<T> dataStream)
StreamTableEnvironment
DataStream
in a given path. Registered views can be
referenced in SQL queries.
See StreamTableEnvironment.fromDataStream(DataStream)
for more information on how a DataStream
is translated into a table.
Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will be inaccessible in the current session. To make the permanent object available again you can drop the corresponding temporary object.
createTemporaryView
in interface StreamTableEnvironment
T
- The type of the DataStream
.path
- The path under which the DataStream
is created. See also the TableEnvironment
class description for the format of the path.dataStream
- The DataStream
out of which to create the view.public <T> void createTemporaryView(String path, DataStream<T> dataStream, Schema schema)
StreamTableEnvironment
DataStream
in a given path. Registered views can be
referenced in SQL queries.
See StreamTableEnvironment.fromDataStream(DataStream, Schema)
for more information on how a DataStream
is translated into a table.
Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will be inaccessible in the current session. To make the permanent object available again you can drop the corresponding temporary object.
createTemporaryView
in interface StreamTableEnvironment
T
- The type of the DataStream
.path
- The path under which the DataStream
is created. See also the TableEnvironment
class description for the format of the path.dataStream
- The DataStream
out of which to create the view.schema
- The customized schema for the final table.public DataStream<Row> toDataStream(Table table)
StreamTableEnvironment
Table
into a DataStream
.
Since the DataStream API does not support changelog processing natively, this method
assumes append-only/insert-only semantics during the table-to-stream conversion. The records
of class Row
will always describe RowKind.INSERT
changes. Updating tables are
not supported by this method and will produce an exception.
If you want to convert the Table
to a specific class or data type, use StreamTableEnvironment.toDataStream(Table, Class)
or StreamTableEnvironment.toDataStream(Table, AbstractDataType)
instead.
Note that the type system of the table ecosystem is richer than the one of the DataStream
API. The table runtime will make sure to properly serialize the output records to the first
operator of the DataStream API. Afterwards, the Types
semantics of the DataStream API
need to be considered.
If the input table contains a single rowtime column, it will be propagated into a stream record's timestamp. Watermarks will be propagated as well.
toDataStream
in interface StreamTableEnvironment
table
- The Table
to convert. It must be insert-only.DataStream
.StreamTableEnvironment.toDataStream(Table, AbstractDataType)
,
StreamTableEnvironment.toChangelogStream(Table)
public <T> DataStream<T> toDataStream(Table table, Class<T> targetClass)
StreamTableEnvironment
Table
into a DataStream
of the given Class
.
See StreamTableEnvironment.toDataStream(Table, AbstractDataType)
for more information on how a Table
is translated into a DataStream
.
This method is a shortcut for:
tableEnv.toDataStream(table, DataTypes.of(targetClass))
Calling this method with a class of Row
will redirect to StreamTableEnvironment.toDataStream(Table)
.
toDataStream
in interface StreamTableEnvironment
T
- External record.table
- The Table
to convert. It must be insert-only.targetClass
- The Class
that decides about the final external representation in
DataStream
records.DataStream
.StreamTableEnvironment.toChangelogStream(Table, Schema)
public <T> DataStream<T> toDataStream(Table table, AbstractDataType<?> targetDataType)
StreamTableEnvironment
Table
into a DataStream
of the given DataType
.
The given DataType
is used to configure the table runtime to convert columns and
internal data structures to the desired representation. The following example shows how to
convert the table columns into the fields of a POJO type.
// given a Table of (name STRING, age INT) public static class MyPojo { public String name; public Integer age; // default constructor for DataStream API public MyPojo() {} // fully assigning constructor for field order in Table API public MyPojo(String name, Integer age) { this.name = name; this.age = age; } } tableEnv.toDataStream(table, DataTypes.of(MyPojo.class));
Since the DataStream API does not support changelog processing natively, this method assumes append-only/insert-only semantics during the table-to-stream conversion. Updating tables are not supported by this method and will produce an exception.
Note that the type system of the table ecosystem is richer than the one of the DataStream
API. The table runtime will make sure to properly serialize the output records to the first
operator of the DataStream API. Afterwards, the Types
semantics of the DataStream API
need to be considered.
If the input table contains a single rowtime column, it will be propagated into a stream record's timestamp. Watermarks will be propagated as well.
toDataStream
in interface StreamTableEnvironment
T
- External record.table
- The Table
to convert. It must be insert-only.targetDataType
- The DataType
that decides about the final external
representation in DataStream
records.DataStream
.StreamTableEnvironment.toDataStream(Table)
,
StreamTableEnvironment.toChangelogStream(Table, Schema)
public DataStream<Row> toChangelogStream(Table table)
StreamTableEnvironment
Table
into a DataStream
of changelog entries.
Compared to StreamTableEnvironment.toDataStream(Table)
, this method produces instances of Row
and sets the RowKind
flag that is contained in every record during runtime. The
runtime behavior is similar to that of a DynamicTableSink
.
This method can emit a changelog containing all kinds of changes (enumerated in RowKind
) that the given updating table requires as the default ChangelogMode
. Use
StreamTableEnvironment.toChangelogStream(Table, Schema, ChangelogMode)
to limit the kinds of changes (e.g.
for upsert mode).
Note that the type system of the table ecosystem is richer than the one of the DataStream
API. The table runtime will make sure to properly serialize the output records to the first
operator of the DataStream API. Afterwards, the Types
semantics of the DataStream API
need to be considered.
If the input table contains a single rowtime column, it will be propagated into a stream record's timestamp. Watermarks will be propagated as well.
toChangelogStream
in interface StreamTableEnvironment
table
- The Table
to convert. It can be updating or insert-only.Row
.public DataStream<Row> toChangelogStream(Table table, Schema targetSchema)
StreamTableEnvironment
Table
into a DataStream
of changelog entries.
Compared to StreamTableEnvironment.toDataStream(Table)
, this method produces instances of Row
and sets the RowKind
flag that is contained in every record during runtime. The
runtime behavior is similar to that of a DynamicTableSink
.
This method can emit a changelog containing all kinds of changes (enumerated in RowKind
) that the given updating table requires as the default ChangelogMode
. Use
StreamTableEnvironment.toChangelogStream(Table, Schema, ChangelogMode)
to limit the kinds of changes (e.g.
for upsert mode).
The given Schema
is used to configure the table runtime to convert columns and
internal data structures to the desired representation. The following example shows how to
convert a table column into a POJO type.
// given a Table of (id BIGINT, payload ROW < name STRING , age INT >) public static class MyPojo { public String name; public Integer age; // default constructor for DataStream API public MyPojo() {} // fully assigning constructor for field order in Table API public MyPojo(String name, Integer age) { this.name = name; this.age = age; } } tableEnv.toChangelogStream( table, Schema.newBuilder() .column("id", DataTypes.BIGINT()) .column("payload", DataTypes.of(MyPojo.class)) // force an implicit conversion .build());
Note that the type system of the table ecosystem is richer than the one of the DataStream
API. The table runtime will make sure to properly serialize the output records to the first
operator of the DataStream API. Afterwards, the Types
semantics of the DataStream API
need to be considered.
If the input table contains a single rowtime column, it will be propagated into a stream record's timestamp. Watermarks will be propagated as well.
If the rowtime should not be a concrete field in the final Row
anymore, or the
schema should be symmetrical for both StreamTableEnvironment.fromChangelogStream(org.apache.flink.streaming.api.datastream.DataStream<org.apache.flink.types.Row>)
and StreamTableEnvironment.toChangelogStream(org.apache.flink.table.api.Table)
, the rowtime can also be declared as a metadata column that will be
propagated into a stream record's timestamp. It is possible to declare a schema without
physical/regular columns. In this case, those columns will be automatically derived and
implicitly put at the beginning of the schema declaration.
The following examples illustrate common schema declarations and their semantics:
// given a Table of (id INT, name STRING, my_rowtime TIMESTAMP_LTZ(3)) // === EXAMPLE 1 === // no physical columns defined, they will be derived automatically, // the last derived physical column will be skipped in favor of the metadata column Schema.newBuilder() .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)") .build() // equal to: CREATE TABLE (id INT, name STRING, rowtime TIMESTAMP_LTZ(3) METADATA) // === EXAMPLE 2 === // physical columns defined, all columns must be defined Schema.newBuilder() .column("id", "INT") .column("name", "STRING") .columnByMetadata("rowtime", "TIMESTAMP_LTZ(3)") .build() // equal to: CREATE TABLE (id INT, name STRING, rowtime TIMESTAMP_LTZ(3) METADATA)
toChangelogStream
in interface StreamTableEnvironment
table
- The Table
to convert. It can be updating or insert-only.targetSchema
- The Schema
that decides about the final external representation
in DataStream
records.Row
.public DataStream<Row> toChangelogStream(Table table, Schema targetSchema, ChangelogMode changelogMode)
StreamTableEnvironment
Table
into a DataStream
of changelog entries.
Compared to StreamTableEnvironment.toDataStream(Table)
, this method produces instances of Row
and sets the RowKind
flag that is contained in every record during runtime. The
runtime behavior is similar to that of a DynamicTableSink
.
This method requires an explicitly declared ChangelogMode
. For example, use ChangelogMode.upsert()
if the stream will not contain RowKind.UPDATE_BEFORE
, or
ChangelogMode.insertOnly()
for non-updating streams.
Note that the type system of the table ecosystem is richer than the one of the DataStream
API. The table runtime will make sure to properly serialize the output records to the first
operator of the DataStream API. Afterwards, the Types
semantics of the DataStream API
need to be considered.
If the input table contains a single rowtime column, it will be propagated into a stream
record's timestamp. Watermarks will be propagated as well. However, it is also possible to
write out the rowtime as a metadata column. See StreamTableEnvironment.toChangelogStream(Table, Schema)
for
more information and examples on how to declare a Schema
.
toChangelogStream
in interface StreamTableEnvironment
table
- The Table
to convert. It can be updating or insert-only.targetSchema
- The Schema
that decides about the final external representation
in DataStream
records.changelogMode
- The required kinds of changes in the result changelog. An exception will
be thrown if the given updating table cannot be represented in this changelog mode.Row
.public StreamStatementSet 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 StreamTableEnvironment
createStatementSet
in interface TableEnvironment
createStatementSet
in class TableEnvironmentImpl
DataStream
APIpublic <T> Table fromDataStream(DataStream<T> dataStream, Expression... fields)
StreamTableEnvironment
DataStream
into a Table
with specified field names.
There are two modes for mapping original fields to the fields of the Table
:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime attributes at arbitrary positions using arbitrary names (except those that exist in the result schema). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataStream<Tuple2<String, Long>> stream = ...
Table table = tableEnv.fromDataStream(
stream,
$("f1"), // reorder and use the original field
$("rowtime").rowtime(), // extract the internally attached timestamp into an event-time
// attribute named 'rowtime'
$("f0").as("name") // reorder and give the original field a better name
);
2. Reference input fields by position: In this mode, fields are simply renamed. Event-time
attributes can replace the field on their position in the input data (if it is of correct
type) or be appended at the end. Proctime attributes must be appended at the end. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataStream<Tuple2<String, Long>> stream = ...
Table table = tableEnv.fromDataStream(
stream,
$("a"), // rename the first field to 'a'
$("b"), // rename the second field to 'b'
$("rowtime").rowtime() // extract the internally attached timestamp into an event-time
// attribute named 'rowtime'
);
fromDataStream
in interface StreamTableEnvironment
T
- The type of the DataStream
.dataStream
- The DataStream
to be converted.fields
- The fields expressions to map original fields of the DataStream to the fields
of the Table
.Table
.public <T> void registerDataStream(String name, DataStream<T> dataStream)
StreamTableEnvironment
DataStream
. Registered views can be referenced in SQL
queries.
The field names of the Table
are automatically derived from the type of the DataStream
.
The view is registered in the namespace of the current catalog and database. To register
the view in a different catalog use StreamTableEnvironment.createTemporaryView(String, DataStream)
.
Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will be inaccessible in the current session. To make the permanent object available again you can drop the corresponding temporary object.
registerDataStream
in interface StreamTableEnvironment
T
- The type of the DataStream
to register.name
- The name under which the DataStream
is registered in the catalog.dataStream
- The DataStream
to register.public <T> void createTemporaryView(String path, DataStream<T> dataStream, Expression... fields)
StreamTableEnvironment
DataStream
in a given path with specified field names.
Registered views can be referenced in SQL queries.
There are two modes for mapping original fields to the fields of the View:
1. Reference input fields by name: All fields in the schema definition are referenced by name (and possibly renamed using an alias (as). Moreover, we can define proctime and rowtime attributes at arbitrary positions using arbitrary names (except those that exist in the result schema). In this mode, fields can be reordered and projected out. This mode can be used for any input type, including POJOs.
Example:
DataStream<Tuple2<String, Long>> stream = ...
tableEnv.createTemporaryView(
"cat.db.myTable",
stream,
$("f1"), // reorder and use the original field
$("rowtime").rowtime(), // extract the internally attached timestamp into an event-time
// attribute named 'rowtime'
$("f0").as("name") // reorder and give the original field a better name
);
2. Reference input fields by position: In this mode, fields are simply renamed. Event-time
attributes can replace the field on their position in the input data (if it is of correct
type) or be appended at the end. Proctime attributes must be appended at the end. This mode
can only be used if the input type has a defined field order (tuple, case class, Row) and
none of the fields
references a field of the input type.
Example:
DataStream<Tuple2<String, Long>> stream = ...
tableEnv.createTemporaryView(
"cat.db.myTable",
stream,
$("a"), // rename the first field to 'a'
$("b"), // rename the second field to 'b'
$("rowtime").rowtime() // adds an event-time attribute named 'rowtime'
);
Temporary objects can shadow permanent ones. If a permanent object in a given path exists, it will be inaccessible in the current session. To make the permanent object available again you can drop the corresponding temporary object.
createTemporaryView
in interface StreamTableEnvironment
T
- The type of the DataStream
.path
- The path under which the DataStream
is created. See also the TableEnvironment
class description for the format of the path.dataStream
- The DataStream
out of which to create the view.fields
- The fields expressions to map original fields of the DataStream to the fields
of the View.public <T> DataStream<T> toAppendStream(Table table, Class<T> clazz)
StreamTableEnvironment
Table
into an append DataStream
of a specified type.
The Table
must only have insert (append) changes. If the Table
is also
modified by update or delete changes, the conversion will fail.
The fields of the Table
are mapped to DataStream
fields as follows:
Row
and Tuple
types: Fields are mapped
by position, field types must match.
DataStream
types: Fields are mapped by field name, field types must match.
toAppendStream
in interface StreamTableEnvironment
T
- The type of the resulting DataStream
.table
- The Table
to convert.clazz
- The class of the type of the resulting DataStream
.DataStream
.public <T> DataStream<T> toAppendStream(Table table, TypeInformation<T> typeInfo)
StreamTableEnvironment
Table
into an append DataStream
of a specified type.
The Table
must only have insert (append) changes. If the Table
is also
modified by update or delete changes, the conversion will fail.
The fields of the Table
are mapped to DataStream
fields as follows:
Row
and Tuple
types: Fields are mapped
by position, field types must match.
DataStream
types: Fields are mapped by field name, field types must match.
toAppendStream
in interface StreamTableEnvironment
T
- The type of the resulting DataStream
.table
- The Table
to convert.typeInfo
- The TypeInformation
that specifies the type of the DataStream
.DataStream
.public <T> DataStream<Tuple2<Boolean,T>> toRetractStream(Table table, Class<T> clazz)
StreamTableEnvironment
Table
into a DataStream
of add and retract messages. The
message will be encoded as Tuple2
. The first field is a Boolean
flag, the
second field holds the record of the specified type T
.
A true Boolean
flag indicates an add message, a false flag indicates a retract
message.
The fields of the Table
are mapped to DataStream
fields as follows:
Row
and Tuple
types: Fields are mapped
by position, field types must match.
DataStream
types: Fields are mapped by field name, field types must match.
toRetractStream
in interface StreamTableEnvironment
T
- The type of the requested record type.table
- The Table
to convert.clazz
- The class of the requested record type.DataStream
.public <T> DataStream<Tuple2<Boolean,T>> toRetractStream(Table table, TypeInformation<T> typeInfo)
StreamTableEnvironment
Table
into a DataStream
of add and retract messages. The
message will be encoded as Tuple2
. The first field is a Boolean
flag, the
second field holds the record of the specified type T
.
A true Boolean
flag indicates an add message, a false flag indicates a retract
message.
The fields of the Table
are mapped to DataStream
fields as follows:
Row
and Tuple
types: Fields are mapped
by position, field types must match.
DataStream
types: Fields are mapped by field name, field types must match.
toRetractStream
in interface StreamTableEnvironment
T
- The type of the requested record type.table
- The Table
to convert.typeInfo
- The TypeInformation
of the requested record type.DataStream
.protected void validateTableSource(TableSource<?> tableSource)
TableEnvironmentImpl
validateTableSource
in class TableEnvironmentImpl
tableSource
- tableSource to validateCopyright © 2014–2024 The Apache Software Foundation. All rights reserved.