Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

...

Page properties


Discussion thread

...

...

...

...

...

thread.html/rf96597f4a6336e985b8d83cfaf3db4e5857b6b15f9fc70f8129d9a5c%40%3Cdev.flink.apache.org%3E

JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-23062

Release1.14



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

...

InterfaceChangeComment
TableEnvironment
#connectRemoveDeprecated 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
ConnectTableDescriptorRemove
BatchTableDescriptorRemove
StreamTableDescriptorRemove
ConnectorDescriptorRemove
TableDescriptorRefactor
  • Removed in its current form
  • Name is re-used for the new API
RowtimeRemove


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
languagejava
titleTableEnvironment
/** creates a 
  * Creates a new table from the given descriptor.
  *
  * 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 descriptor. 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 tablePathpath, TableDescriptor tableDescriptordescriptor);


The TableDescriptor interface 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:

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
languagejava
titleTableDescriptor
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
languagejava
tEnv.createTable(
  "cat.db.MyTable",

  TableDescriptor.forConnector("kafka")
    .comment("This is a comment")
    .schema(Schema.newBuilder()
      .column("f0
Code Block
languagejava
// register a table using specific descriptor
tEnv.createTemporaryTable(
	"MyTable",
	new Kafka()
		.version("0.11")
		.topic("user_logs")
		.property("bootstrap.servers", "localhost:9092")
		.property("group.id", "test-group")
		.startFromEarliest()
		.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.STRING())
				.column("part_field_0", DataTypes.STRING())
				.column("part_field_1", DataTypes.INT())
				.proctime("proc") // define a processing-time attribute with column name "proc"
				.computedColumn("my_ts", "TO_TIMESTAMP(log_ts)")  // computed column
				.watermarkFor("my_ts").boundedOutOfOrderTimestamps(Duration.ofSeconds(5))  // defines watermark and rowtime attribute
				.primaryKey("user_id"))
		.partitionedBy("part_field_0", "part_field_1")  // Kafka doesn't support partitioned table yet, this is just an example for the API
);

// register a table using general purpose Connector descriptor, this would be helpful for custom source/sinks
tEnv.createTemporaryTable(
	"MyTable",
	new Connector("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(
			new Schema()
				.column("user_id", DataTypes.BIGINT())
				.column("user_name", DataTypes.STRING())
				.column("score      .columnByExpression("f1", "2 * f0")
      .columnByMetadata("f3", DataTypes.DECIMALSTRING(10, 2))
				      .column("log_tst", DataTypes.STRINGTIMESTAMP(3))
				.column("part_field_0", DataTypes.STRING())
				.column("part_field_1", DataTypes.INT())
				.proctime("proc") // define a processing-time attribute with column name "proc"
				.computedColumn("my_ts", "TO_TIMESTAMP(log_ts)")  // computed column
				.watermarkFor("my_ts").boundedOutOfOrderTimestamps(Duration.ofSeconds(5)) // defines watermark and rowtime attribute
				.primaryKey("user_id"))
		.partitionedBy("part_field_0", "part_field_1") // Kafka doesn't support partitioned table yet, this is just an example for the API
);      .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
languagejava
titleTableEnvironment
/**interface TableEnvironment {
  /** readsReturns a table from {@link Table} backed by the given descriptor.
  */
  Table from(TableDescriptor tableDescriptordescriptor); 

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
languagejava
titleTable
/**
  // 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. */
   * 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, 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
languagejava
/**
  * 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
languagejava
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", 

...

Briefly list any new interfaces that will be introduced as part of this proposal or any existing interfaces that will be removed or changed. The purpose of this section is to concisely call out the public contract that will come along with this feature.

A public interface is any change to the following:

  • Binary log format

  • The network protocol and api behavior

  • Any class in the public packages under clientsConfiguration, especially client configuration

    • org/apache/kafka/common/serialization

    • org/apache/kafka/common

    • org/apache/kafka/common/errors

    • org/apache/kafka/clients/producer

    • org/apache/kafka/clients/consumer (eventually, once stable)

  • Monitoring

  • Command line tools and arguments

  • Anything else that will likely break existing users in some way when they upgrade

Proposed Changes

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

...

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

  1. 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".
  2. 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()".
  3. 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.