Versions Compared

Key

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

Status

...

Page properties


Discussion thread

...

...

...

...

Voting thread: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/VOTE-FLIP-129-Refactor-Descriptor-API-to-register-connector-in-Table-API-td43420.html

...

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 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"), 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-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()

Code Block
languagejava
/** 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:

Code Block
languagejava
// 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()
);
Code Block
languagejava
// 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:

Code Block
languagejava
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:

Code Block
languagejava
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. 

Code Block
languagejava
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:

Code Block
languagejava
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:

Code Block
languagejava
titleTableDescriptor
/**
 * 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;
	}
}
Code Block
languagejava
titleLikeOption
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
	}
}
Code Block
languagejava
titleConnector
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;
		}
	}
}
Code Block
languagejava
titleKafka
/**
 * 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

Code Block
languagejava
/**
 * 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:

Code Block
languagejava
titleSchema
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/SchemaTableDescriptorRegistration 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

...

ConnectTableDescriptor API Changes

...

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 methods:

Code Block
languagejava
titleTableEnvironment
/**
  * 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 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
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", 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).

Code Block
languagejava
titleTableEnvironment
/**
  * Returns a {@link Table} backed by the given descriptor.
  */
Table from(TableDescriptor 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
languagejava
titleTable
/**
  * 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 descriptor);

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

...