Status
Current state: Accepted
Discussion thread: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-129-Refactor-Descriptor-API-to-register-connector-in-Table-API-tp42995.html
JIRA:
Released: <Flink Version>
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Flink introduced Descriptor API to configure and instatiate TableSources/TableSinks since 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.
- 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 implement a corresponding Descriptor (e.g. new Kafka()) to use the "connect" API. 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.
- The underlying implementation of Descriptor API and SQL DDL are different. It’s expensive to maintain two different code path.
- There are many known issues about Descriptor API: FLINK-17548, FLINK-17186, FLINK-15801, FLINK-15943.
Public Interfaces
We propose to drop the existing method TableEnvironment#connect
(deprecated in 1.11) and some related interfaces/classes, including:
(drop) TableEnvironment#connect
(drop) ConnectTableDescriptor
(drop) BatchTableDescriptor
(drop) StreamTableDescriptor
(drop) ConnectorDescriptor
(drop) Rowtime
(refactor) TableDescriptor
(refactor) Schema
We propose to introduce a new set of descriptor APIs for Table API.
TableEnvironment#createTemporaryTable()
/** creates a temporary table from a descriptor. */ void createTemporaryTable(String tablePath, TableDescriptor 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:
// register a table using specific descriptor tEnv.createTemporaryTable( "MyTable", KafkaConnector.newBuilder() .version("0.11") .topic("user_logs") .property("bootstrap.servers", "localhost:9092") .property("group.id", "test-group") .scanStartupModeEarliest() .sinkPartitionerRoundRobin() .format(JsonFormat.newBuilder().ignoreParseErrors(false).build()) .schema( Schema.newBuilder() .column("user_id", DataTypes.BIGINT()) .column("user_name", DataTypes.STRING()) .column("score", DataTypes.DECIMAL(10, 2)) .column("log_ts", DataTypes.STRING()) .column("part_field_0", DataTypes.STRING()) .column("part_field_1", DataTypes.INT()) .column("proc", proctime()) // define a processing-time attribute with column name "proc" .column("ts", toTimestamp($("log_ts"))) .watermark("ts", $("ts").minus(lit(3).seconds())) .primaryKey("user_id") .build()) .partitionedBy("part_field_0", "part_field_1") // Kafka doesn't support partitioned table yet, this is just an example for the API .build() );
// register a table using general purpose Connector descriptor, this would be helpful for custom source/sinks tEnv.createTemporaryTable( "MyTable", Connector.of("kafka-0.11") .option("topic", "user_logs") .option("properties.bootstrap.servers", "localhost:9092") .option("properties.group.id", "test-group") .option("scan.startup.mode", "earliest") .option("format", "json") .option("json.ignore-parse-errors", "true") .option("sink.partitioner", "round-robin") .schema( Schema.newBuilder() .column("user_id", DataTypes.BIGINT()) .column("user_name", DataTypes.STRING()) .column("score", DataTypes.DECIMAL(10, 2)) .column("log_ts", DataTypes.STRING()) .column("part_field_0", DataTypes.STRING()) .column("part_field_1", DataTypes.INT()) .column("proc", proctime()) // define a processing-time attribute with column name "proc" .column("ts", toTimestamp($("log_ts"))) .watermark("ts", $("ts").minus(lit(3).seconds())) .primaryKey("user_id") .build()) .partitionedBy("part_field_0", "part_field_1") // Kafka doesn't support partitioned table yet, this is just an example for the API .build() );
LIKE clause for Descriptor API
We propose to support .like(...)
method on the TableDescriptor to support the same functionality of LIKE clause in CREATE TABLE DDL (FLIP-110). You can refer to FLIP-110 for more details about like options.
Here is a simple example to derive table from existing one:
tEnv.createTemporaryTable( "OrdersInKafka", KafkaConnector.newBuilder() .topic("user_logs") .property("bootstrap.servers", "localhost:9092") .property("group.id", "test-group") .format(JsonFormat.newInstance()) .schema( Schema.newBuilder() .column("user_id", DataTypes.BIGINT()) .column("score", DataTypes.DECIMAL(10, 2)) .column("log_ts", DataTypes.TIMESTAMP(3)) .column("ts", toTimestamp($("log_ts"))) .build()) .build() ); tEnv.createTemporaryTable( "OrdersInFilesystem", Connector.of("filesystem") .option("path", "path/to/whatever") .schema( Schema.newBuilder() .watermark("ts", $("ts").minus(lit(3).seconds())) .build()) .like("OrdersInKafka", LikeOption.EXCLUDING.ALL, LikeOption.INCLUDING.GENERATED) .build() );
The above "OrdersInFilesystem" table will be equivalent to:
tEnv.createTemporaryTable( "OrdersInFilesystem", Connector.of("filesystem") .option("path", "path/to/whatever") .schema( Schema.newBuilder() .column("user_id", DataTypes.BIGINT()) .column("score", DataTypes.DECIMAL(10, 2)) .column("log_ts", DataTypes.TIMESTAMP(3)) .column("my_ts", $("ts")) .build()) .build() );
TableEnvironment#from() and Table#executeInsert()
Additionally, we would like to propose two new methods for better usability for Table API users.
interface TableEnvironment { /** reads a table from the given descriptor */ Table from(TableDescriptor tableDescriptor); // 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. */ TableResult executeInsert(TableDescriptor tableDescriptor); // 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 registration step and can use the source/sink out-of-box. For example:
Schema schema = Schema.newBuilder() .column("user_id", DataTypes.BIGINT()) .column("score", DataTypes.DECIMAL(10, 2)) .column("ts", DataTypes.TIMESTAMP(3)) .build(); Table myKafka = tEnv.from( KafkaConnector.newBuilder() .version("0.11") .topic("user_logs") .property("bootstrap.servers", "localhost:9092") .property("group.id", "test-group") .scanStartupModeEarliest() .sinkPartitionerRoundRobin() .format(JsonFormat.newBuilder().ignoreParseErrors(false).build()) .schema(schema) .build() ); // reading from kafka table and write into filesystem table myKafka.executeInsert( Connector.of("filesystem") .option("path", "/path/to/whatever") .option("format", "json") .schema(schema) .build() );
Proposed Changes
We will discuss in detail about the new interfaces/classes in this section.
All the classes will be located in org.apache.flink.table.descriptors
package and in flink-table-common
module.
TableDescriptor
is an abstract class, it represents a SQL DDL strucutre or a CatalogTable. It can be divided into several parts: schema, partitionedKey, and options. The TableDescriptor
determines how to define schema and partitionedKeys, but leaves options to be implemented by subclasses. Specific connectors can extend to TableDescriptor
and provide handy methods to set connector options (e.g. Kafka#topic(..)
). We also propose to provide a built-in and general implementation of TableDescriptor
, i.e. Connector
. The Connector class provides a general option(String key, String value)
method, thus it can support arbitrary custom connector implementations (based on FLIP-95). The Connector class can reduce the effort of development of custom connectors without implementing a specific descriptor.
TableDescriptor & TableDescriptorBuilder
The current TableDescriptor
will be refactored into:
/** * Describes a table to connect. It is a same representation of SQL CREATE TABLE DDL. It wraps the needed meta information about a catalog table. * Please use a specific {@link TableDescriptorBuilder} to build the {@link TableDescriptor}. */ @PublicEvolving public interface TableDescriptor { List<String> getPartitionedFields(); Schema getSchema(); Map<String, String> getOptions(); LikeOption[] getLikeOptions(); String getLikePath(); } /** * A basic builder implementation to build a {@link TableDescriptor}. */ @PublicEvolving public abstract class TableDescriptorBuilder<BUILDER extends TableDescriptorBuilder<BUILDER>> { private final InternalTableDescriptor descriptor = new InternalTableDescriptor(); /** * Returns the this builder instance in the type of subclass. */ protected abstract BUILDER self(); /** * Specifies the table schema. */ public BUILDER schema(Schema schema) { descriptor.schema = schema; return self(); } /** * Specifies the partition keys of this table. */ public BUILDER partitionedBy(String... fieldNames) { checkArgument(descriptor.partitionedFields.isEmpty(), "partitionedBy(...) shouldn't be called more than once."); descriptor.partitionedFields.addAll(Arrays.asList(fieldNames)); return self(); } /** * Extends some parts from the original registered table path. */ public BUILDER like(String tablePath, LikeOption... likeOptions) { descriptor.likePath = tablePath; descriptor.likeOptions = likeOptions; return self(); } protected BUILDER option(String key, String value) { descriptor.options.put(key, value); return self(); } /** * Returns created table descriptor. */ public TableDescriptor build() { return descriptor; } }
public 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 } }
public class Connector { public static ConnectorBuilder of(String identifier) { return new ConnectorBuilder(identifier); } public static class ConnectorBuilder extends TableDescriptorBuilder<ConnectorBuilder> { private ConnectorBuilder(String identifier) { option(CONNECTOR.key(), identifier); } public ConnectorBuilder option(String key, String value) { String lowerKey = key.toLowerCase().trim(); if (CONNECTOR.key().equals(lowerKey)) { throw new IllegalArgumentException("It's not allowed to override 'connector' option."); } return super.option(key, value); } @Override protected ConnectorBuilder self() { return this; } } }
/** * Connector descriptor for the Apache Kafka message queue. */ @PublicEvolving public class KafkaConnector { public static KafkaConnectorBuilder newBuilder() { return new KafkaConnectorBuilder(); } public static class KafkaConnectorBuilder extends TableDescriptorBuilder<KafkaConnectorBuilder> { /** * Sets the Kafka version to be used. * * @param version Kafka version. E.g., "0.8", "0.11", etc. */ public KafkaConnectorBuilder version(String version) { Preconditions.checkNotNull(version); option(CONNECTOR.key(), "kafka-" + version); return this; } /** * Sets the topic from which the table is read. * * @param topic The topic from which the table is read. */ public KafkaConnectorBuilder topic(String topic) { Preconditions.checkNotNull(topic); option("topic", topic); return this; } // TODO: add more methods ... @Override protected KafkaConnectorBuilder self() { return this; } } }
NOTE: when we refactoring classes such as KafkaConnector
or ElasticsearchConnector
, we should align the method names with the new property names introduced in FLIP-122. For example, the property method for property scan.startup.mode=earliest-offset
should be scanStartupModeEarliest
, instead of startFromEarliest
.
FormatDescriptor & FormatDescriptorBuilder
/** * Describes the format of data. * Please use a specific {@link FormatDescriptorBuilder} to build a {@link FormatDescriptor}. */ @PublicEvolving public interface FormatDescriptor { /** * Converts this descriptor into a set of connector options. */ Map<String, String> toFormatOptions(); } /** * A basic builder implementation to build {@link FormatDescriptor}. */ @PublicEvolving public abstract class FormatDescriptorBuilder<BUILDER extends FormatDescriptorBuilder<BUILDER>> { private final InternalFormatDescriptor descriptor = new InternalFormatDescriptor(); protected abstract BUILDER self(); protected BUILDER option(String key, String value) { descriptor.formatOptions.put(key, value); return self(); } public FormatDescriptor build() { return descriptor; } } /** * Format descriptor for JSON. */ public class JsonFormat { public static FormatDescriptor newInstance() { return newBuilder().build(); } public static JsonFormatBuilder newBuilder() { return new JsonFormatBuilder(); } public static class JsonFormatBuilder extends FormatDescriptorBuilder<JsonFormatBuilder> { /** * Sets flag whether to fail if a field is missing or not. * * @param failOnMissingField If set to true, the operation fails if there is a missing field. * If set to false, a missing field is set to null. */ public JsonFormatBuilder failOnMissingField(boolean failOnMissingField) { option("fail-on-missing-field", String.valueOf(failOnMissingField)); return this; } /** * Sets flag whether to fail when parsing json fails. * * @param ignoreParseErrors If set to true, the operation will ignore parse errors. * If set to false, the operation fails when parsing json fails. */ public JsonFormatBuilder ignoreParseErrors(boolean ignoreParseErrors) { option("ignore-parse-errors", String.valueOf(ignoreParseErrors)); return this; } @Override protected JsonFormatBuilder self() { return this; } } }
Schema
The current Rowtime
class will be removed. The current Schema
class will be refactored into:
public class Schema { // package visible variables used to build TableSchema protected final List<Column> columns; protected final WatermarkInfo watermarkInfo; protected final List<String> primaryKey; private Schema(List<Column> columns, WatermarkInfo watermarkInfo, List<String> primaryKey) { this.columns = columns; this.watermarkInfo = watermarkInfo; this.primaryKey = primaryKey; } public static SchemaBuilder newBuilder() { return new SchemaBuilder(); } public static class SchemaBuilder { List<Column> columns = new ArrayList<>(); WatermarkInfo watermarkInfo; List<String> primaryKey; private SchemaBuilder() { } /** * Adds a column with the column name and the data type. */ public SchemaBuilder column(String fieldName, AbstractDataType<?> fieldType) { columns.add(new PhysicalColumn(fieldName, fieldType)); return this; } public SchemaBuilder column(String fieldName, Expression expr) { columns.add(new VirtualColumn(fieldName, expr)); return this; } public SchemaBuilder primaryKey(String... fieldNames) { this.primaryKey = Arrays.asList(fieldNames); return this; } public SchemaBuilder watermark(String rowtimeField, Expression watermarkExpr) { this.watermarkInfo = new WatermarkInfo(rowtimeField, watermarkExpr); return this; } public Schema build() { return new Schema(columns, watermarkInfo, primaryKey); } }
Implementation
I propose to only support this in blink planner as we are going to drop old planner in the near future and old planner doesn't support FLIP-95 connectors.
The descriptors TableDescriptor/Schema
can be used in TableEnvironment#from()
, TableEnvironment#createTemporaryTable()
, Table#executeInsert()
. So does the StreamTableEnvironment
, but BatchTableEnvironment
will not support this as it is implemented in old planner.
The descriptors TableDescriptor/Schema
only defines the meta information (just like DDL string) used to build the CatalogTable. The implementation of TableEnvironment#createTemporaryTable(path, descriptor) will translate the descriptor into CatalogTable.
TableDescriptor
stores the meta information in the package-visible member fields/methods, e.g. schema, partitionedKeys, options
, so does the Schema
class.
TableEnvironmentImpl#createTemporaryTable will create a new instance of TableDescriptorRegistration
to register descriptor as a CatalogTable into catalog. It is an @Internal class located in org.apache.flink.table.descriptors.
So that TableDescriptorRegistration
can access member fields in TableDescriptor/Schema
. TableDescriptorRegistration
will convert schema
into TableSchema
(with the help of CatalogTableSchemaResolver
), and convert partitionedKeys, options, tableSchema
into CatalogTableImpl
.
TableEnvironment#from(descriptor)
will create a specific QueryOperation and will be converted to scan RelNode later in QueryOperationConverter. Table#executeInsert()
will create a specific ModifyOperation and will be converted to sink RelNode later in PlannerBase.
Compatibility, Deprecation, and Migration Plan
This is indeed an incompatible interface change, because we propose to drop the existing one and introduce new ones. But I think this is fine, as TableEnvironment#connect
has been deprecated in 1.11. For the users who are using TableEnvironment#connect in 1.11, we have recommended them in the Javadoc to use SQL CREATE TABLE DDL (TableEnvironment#executeSql(String)
) instead.
We have a migration plan for users who are still using TableEnvironment#connect
and want to migrate to new Descriptor APIs. The following tables list the API changes:
Schema API Changes
Current Interface | New Interface |
---|---|
.field("name", DataTypes.STRING()) | .column("name", DataTypes.STRING()) |
.field("name", "STRING") | .column("name", DataTypes.STRING()) |
.field("proctime", DataTypes.TIMESTAMP(3)).proctime() | .column("proc", proctime()) |
.from("originalName") | .column("newName", $("originalName")) |
.rowtime(new Rowtime().timestampsFromField("time").watermarksPeriodicAscending()) | .watermarkFor("time", $("time")) |
.rowtime(new Rowtime().timestampsFromField("time").watermarksPeriodicBounded(2)) | .watermark("time", $("time").minus(lit(2).milli())) |
rowtime.timestampsFromExtractor(TimestampExtractor) | .column(fieldName, expr).watermark(fieldName, watermarkExpr) |
rowtime.watermarksFromStrategy(WatermarkStrategy) | .watermark(fieldName, watermarkExpr) |
rowtime.watermarksFromSource() | removed (never implemented by any connectors before) |
rowtime.timestampsFromSource() | removed (never implemented by any connectors before) |
.toProperties() | removed (not needed anymore) |
ConnectTableDescriptor API Changes
Current Interface | New Interface |
---|---|
.withSchema(new Schema()) | .schema(new Schema()) |
.withFormat(new Json()) | KafkaConnector.newBuilder.format(JsonFormat.newInstance()).build() |
.withPartitionKeys(Arrays.asList(a, b, c)) | .partitionedBy(a, b, c) |
.createTemporaryTable(path) | tEnv.createTemporaryTable(path, descriptor) |
.inAppendMode() | removed (not needed anymore) |
.inRetractMode() | removed (not needed anymore) |
.inUpsertMode() | removed (not needed anymore) |
.toProperties() | removed (not needed anymore) |
Rejected Alternatives
Keep and follow the original TableEnvironment#connect
API
For example, a minor refactored TableEnvironment#connect:
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", 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.