Status
...
Page properties | |
---|---|
|
...
...
...
...
...
|
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Table of Contents |
---|
Motivation
Flink introduced Descriptor API to configure and instatiate TableSources/TableSinks since The TableEnvironment#connect API has been introduced in Flink 1.5.0 , i.e. the TableEnvironment#connect
API.
Currently, there are many problems with current Descriptor API which we want to resolve in this FLIP.
in order to instantiate and configure table sources and sinks. Since then, the SQL DDL has been actively developed and improved, and as a result is more powerful and many of these feature are inaccessible from #connect. Furthermore, this API has shown to contain several shortcomings:
- Connectors have to implement corresponding descriptors
- The community focused on the new SQL DDL feature in recent releases. SQL DDL is well-designed and has many rich features. However, Descriptor API lacks many key features, e.g. computed columns, primary key, partition key and so on.
- Currently, a connector must provide a corresponding Descriptor (e.g. "new Kafka()). We hope connectors can be registered without a corresponding Descriptor. This can ease the development of connectors and can be a replacement of registerTableSource/Sink"), which increases maintenance effort and duplicates information.
- The underlying implementation of Descriptor API and for the SQL DDL and #connector are different. It’s expensive to maintain two , requiring the maintenance of different code pathpaths.
- There are many known issues about Descriptor API: FLINK-
...
As a result, #connect has been deprecated sind Flink 1.11. In this FLIP, we want to propose a new API to programmatically define sources and sinks on the Table API without having to switch to SQL DDL.
Public Interfaces
...
Interface | Change | Comment |
---|---|---|
TableEnvironment | ||
#connect | Remove | Deprecated since Flink 1.11 |
#createTable(path, TableDescriptor) | New | |
#createTemporaryTable(path, TableDescriptor) | New | |
#from(TableDescriptor) | New | |
Table | ||
#executeInsert(TableDescriptor) | New | |
StatementSet | ||
#addInsert(TableDescriptor, Table) | New | |
Other | ||
ConnectTableDescriptor | Remove | |
BatchTableDescriptor | Remove | |
StreamTableDescriptor | Remove | |
ConnectorDescriptor | Remove | |
TableDescriptor | Refactor |
|
Rowtime | Remove |
TableEnvironment#createTable & TableEnvironment#createTemporaryTable
In order for users to register sources and sinks via Table API, we introduce two new methodsWe propose to drop the existing method TableEnvironment#connect
(deprecated in 1.11) and introduce a new method in TableEnvironment
:
Code Block | ||||
---|---|---|---|---|
| ||||
/** * createsCreates a temporarynew table from the agiven descriptor. */ void createTemporaryTable(String tablePath, TableDe scriptor tableDescriptor); |
The TableDescriptor
is an unified interface/class to represent a SQL DDL strucutre (or CatalogTable internally). It can be a specific connector descriptor, e.g. Kafka
, or a general purpose descriptor, i.e. Connector
. All the methods can be chained called on the instance, including .schema()
, .partitionedBy()
, and .like()
. We will discuss TableDescriptor
in detail in the Proposed Changes section.
A full example will look like this:
...
* The table is created in the catalog defined by the given path.
*/
void createTable(String path, TableDescriptor descriptor);
/**
* Creates a new temporary table from the given descriptor.
*
* 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.
*/
void createTemporaryTable(String path, TableDescriptor descriptor); |
The TableDescriptor interface is a (generic) representation of the structure used in the SQL DDL, or CatalogTable, respectively. It implements a fluent API to allow chaining and make it easy to use. Options are either specified by referring to an actual ConfigOption instance (preferred), or by string. The latter is necessary, in particular, for options which are not represented through ConfigOption instances, e.g. if they contain placeholders such as "field.#.min". The interface also offers quality-of-life methods for specifying formats such that prefixing format options is handled by the descriptor itself, which allows using ConfigOption instances for format options.
Code Block | ||||
---|---|---|---|---|
| ||||
TableDescriptor {
// Create a builder
static TableDescriptorBuilder forConnector(String connector);
Optional<Schema> getSchema();
Map<String, String> getOptions();
Optional<String> getComment();
List<String> getPartitionKeys();
Optional<TableLikeDescriptor> getLikeDescriptor();
}
TableDescriptorBuilder<SELF> {
SELF schema(Schema schema);
SELF comment(String comment);
SELF option<T>(ConfigOption<T> configOption, T value);
SELF option(String key, String value);
SELF format(String format);
SELF format(ConfigOption<?> formatOption, String format);
SELF format(FormatDescriptor formatDescriptor);
SELF format(ConfigOption<?> formatOption, FormatDescriptor formatDescriptor);
SELF partitionedBy(String... partitionKeys);
SELF like(String tableName, LikeOption... likeOptions);
TableDescriptor build();
}
TableLikeDescriptor {
String getTableName();
List<TableLikeOption> getLikeOptions();
}
FormatDescriptor {
static FormatDescriptorBuilder forFormat(String format);
String getFormat();
Map<String, String> getOptions();
}
FormatDescriptorBuilder<SELF> {
SELF option<T>(ConfigOption<T> configOption, T value);
SELF option(String key, String value);
FormatDescriptor build();
}
interface LikeOption {
enum INCLUDING implements LikeOption {
ALL,
CONSTRAINTS,
GENERATED,
OPTIONS,
PARTITIONS,
WATERMARKS
}
enum EXCLUDING implements LikeOption {
ALL,
CONSTRAINTS,
GENERATED,
OPTIONS,
PARTITIONS,
WATERMARKS
}
enum OVERWRITING implements LikeOption {
GENERATED,
OPTIONS,
WATERMARKS
}
} |
The following example demonstrates a simple example of how these APIs can be used:
Code Block | ||
---|---|---|
| ||
tEnv.createTable( "cat.db.MyTable", TableDescriptor.forConnector("kafka") .comment("This is a comment") .schema(Schema.newBuilder() .column(" |
...
f0", DataTypes.BIGINT()) |
...
...
. |
...
columnByExpression(" |
...
f1", |
...
"2 * f0") . |
...
columnByMetadata(" |
...
f3", DataTypes. |
...
STRING( |
...
)) |
...
.column(" |
...
t", DataTypes. |
...
TIMESTAMP(3)) |
...
...
. |
...
watermark(" |
...
t", |
...
"t - INTERVAL '1' MINUTE") . |
...
primaryKey(" |
...
f0") .build()) . |
...
partitionedBy(" |
...
f0") |
...
...
.option(KafkaOptions.TOPIC, topic) |
...
. |
...
option(" |
...
properties.bootstrap.servers", " |
...
…") |
...
|
...
.format("json") . |
...
build() ); tEnv.createTemporaryTable( "MyTemporaryTable", TableDescriptor.forConnector("kafka") // … . |
...
like(" |
...
cat.db.MyTable")
); |
TableEnvironment#from
We propose introducing TableEnvironment#from in order to create a Table from a given descriptor. This is in line with existing methods, e.g. from(String).
...
Additionally, we would like to propose two new methods for better usability for Table API users.
Code Block | ||||
---|---|---|---|---|
| ||||
/**interface TableEnvironment { /** readsReturns a table from{@link Table} backed by the given descriptor. */ Table from(TableDescriptor tableDescriptordescriptor); // we already have a "from(String)" method to get registered table from catalog } interface Table { /** Writes the Table to a sink that is specified by the given descriptor. */ |
Table#executeInsert
We propose introducing Table#executeInsert in order to write directly to a sink defined by a descriptor. This is in line with existing methods, e.g. executeInsert(String).
Code Block | ||||
---|---|---|---|---|
| ||||
/** * Declares that the pipeline defined by this table should be written to a table defined by the given descriptor. * * If no schema is defined in the descriptor, it will be inferred automatically. */ TableResult executeInsert(TableDescriptor tableDescriptordescriptor); // we already have a "executeInsert(String)" method to write into a registered table in catalog } |
With the above two methods, we can leverage the same TableDescriptor
definition, then Table API users can skip the table registering step and can use the source/sink out-of-box. For example:
...
StatementSet#addInsert
Similarly to Table#executeInsert, we propose extending StatementSet#addInsert to take a descriptor.
Code Block | ||
---|---|---|
| ||
/**
* Adds the given table as a sink defined by the descriptor.
*/
StatementSet addInsert(TableDescriptor descriptor, Table table); |
Package Structure
The new descriptor API will reside in flink-table-api-java in the org.apache.flink.table.descriptors package.
In order to make discovery of ConfigOption instances easier, we propose to move *Options classes from all built-in connectors (e.g. KafkaOptions, …) into a common package. Then users can easily discover those classes for any connectors on their current classpath. As this involves declaring these classes to be public (evolving) API, minor refactorings will be necessary to ensure they do not contain any internals.
Compatibility, Deprecation, and Migration Plan
The proposed changes drop an existing API and replace it with a new, incompatible API. However, the old API has been deprecated since Flink 1.11 and is lacking support for many features, thus we are not expecting this to affect many users, as the current recommendation has been to switch to SQL DDL. For affected users, the migration requires relatively small changes and all existing features can be covered.
Rejected Alternatives
Keep and follow the original TableEnvironment#connect
API
For example, a minor refactored TableEnvironment#connect:
Code Block | ||
---|---|---|
| ||
tableEnv.connect( new Kafka() // can be replaced by new Connector("kafka-0.11") .version("0.11") |
...
.topic(" |
...
myTopic") |
...
.property("bootstrap.servers", "localhost:9092") |
...
.property("group.id", "test-group") |
...
. |
...
scanStartupModeEarliest() |
...
.sinkPartitionerRoundRobin() |
...
.format(new Json().ignoreParseErrors(false)) |
...
...
.schema( |
...
new |
...
Schema( |
...
) |
...
...
|
...
. |
...
column(" |
...
user_id", |
...
Proposed Changes
TableDescriptor
The current TableDescriptor
will be refactored into:
Code Block | ||||
---|---|---|---|---|
| ||||
/**
* Describes a table to connect. It is a same representation of SQL CREATE TABLE DDL.
*/
@PublicEvolving
public abstract class TableDesc
riptor {
/**
* Specifies the table schema.
*/
public TableDescriptor schema(Schema schema) {...}
/**
* Specifies the partition keys of this table.
*/
public TableDescriptor partitionedBy(String... fieldNames) {...}
/**
* Extends some parts from the original table.
*/
public TableDescriptor like(String originalTablePath, LikeOption... likeOptions) {...}
/**
* Extends some parts from the original table.
*/
public TableDescriptor like(TableDescriptor originalTableDescriptor, LikeOption... likeOptions) {...}
/**
* Specifies the connector options of this table, subclasses should override this method.
*/
protected abstract Map<String, String> connectorOptions();
} |
Schema
The current Rowtime
class will be removed. The current Schema
class will be refactored into:
Code Block | ||||
---|---|---|---|---|
| ||||
public class Schema {
/**
* Adds a column with the column name and the data type.
*/
public Schema column(String columnName, DataType columnType) {...}
/**
* Adds a computed column with the column name and the SQL expression string.
*/
public Schema computedColumn(String columnName, String sqlExpression) {...}
/**
* Adds a processing-time column with the given column name.
*/
public Schema proctime(String columnName) {...}
/**
* Specifies the primary key constraint for a set of given columns.
*/
public Schema primaryKey(String... columnNames) {...}
/**
* Specifies the watermark strategy for rowtime attribute.
*/
public SchemaWithWatermark watermarkFor(String rowtimeColumn) {...}
public static class SchemaWithWatermark {
/**
* Specifies a custom watermark strategy using the given SQL expression string.
*/
public Schema as(String watermarkSqlExpr) {...}
/**
* Specifies a watermark strategy for situations with monotonously ascending timestamps.
*/
public Schema ascendingTimestamps() {...}
/**
* Specifies a watermark strategy for situations where records are out of order, but you can place
* an upper bound on how far the events are out of order. An out-of-order bound B means that
* once the an event with timestamp T was encountered, no events older than {@code T - B} will
* follow any more.
*/
public Schema boundedOutOfOrderTimestamps(Duration maxOutOfOrderness) {...}
}
} |
Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.
Compatibility, Deprecation, and Migration Plan
- What impact (if any) will there be on existing users?
- If we are changing behavior how will we phase out the older behavior?
- If we need special migration tools, describe them here.
- When will we remove the existing behavior?
Test Plan
Describe in few sentences how the FLIP will be tested. We are mostly interested in system tests (since unit-tests are specific to implementation details). How will we know that the implementation works as expected? How will we know nothing broke?
Rejected Alternatives
...
DataTypes.BIGINT())
.column("user_name", DataTypes.STRING())
.column("score", DataTypes.DECIMAL(10, 2))
.column("log_ts", DataTypes.TIMESTAMP(3))
.column("part_field_0", DataTypes.STRING())
.column("part_field_1", DataTypes.INT())
.column("proc", proctime())
.column("my_ts", toTimestamp($("log_ts"))
.watermarkFor("my_ts", $("my_ts").minus(lit(3).seconds())))
.primaryKey("user_id")
.partitionedBy("part_field_0", "part_field_1")
.createTemporaryTable("MyTable"); |
However, we prefer "TableEnvironment#createTemporaryTable(path, descriptor)" instead of "TableEnvironment#connect", because
- It may confuse users that the "connect()" method invoking doesn't connect to external system, it's just a start point to connect to external system. It is connected after the invoking of "createTemporaryTable".
- The "connect()" method looks weired in the methods of TableEnvironment, because all the other methods are SQL compliant. Thus, we think "tEnv#createTemporaryTable(path, descriptor)" is a better entry point than "connect()".
- The "TableEnvironment#createTemporaryTable(path, descriptor)" decouples Descriptor and table registration. We can easily support more features, like "TableEnvironment#from(descriptor)" and "Table#executeInsert(descriptor)" with the same descriptor interfaces/classes.