Versions Compared

Key

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

...

Public Interfaces

We propose to deWe propose to drop the existing method TableEnvironment#connect (deprecated in 1.11) and some related interfaces/classes, including:

...

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);

...

A full example will look like this:

// 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")
);
tEnv.createTemporaryTable(
"MyTable",
new Connector("kafka-0.11")
.option("topic", "user_logs")
.option("properties.bootstrap.servers", "localhost:9092 // 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("properties.group.idtopic", "test-groupuser_logs")
.option("scanproperties.startup.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", 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
);

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 registering step and can use the source/sink out-of-box. For example:


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",
new Kafka()
.topic("user_logs")
.property("bootstrap.servers", "localhost:9092")
.property("group.id", "test-group")
.format(new Json().ignoreParseErrors(false))
.schema(
new Schema()
.column("user_id", DataTypes.BIGINT())
Schema schema = new Schema()
.column("user_idscore", DataTypes.BIGINTDECIMAL(10, 2))
.column("score", DataTypes.DECIMAL(10, 2))
.column("log_ts", DataTypes.TIMESTAMP(3));
Table myKafka = tEnv.from(
new Kafka()
.versioncomputedColumn("0.11"my_ts", "TO_TIMESTAMP(log_ts)")
.topic("user_logs"))
);

tEnv.createTemporaryTable(
"OrdersInFilesystem",
new .propertyConnector("bootstrap.servers", "localhost:9092filesystem")
.propertyoption("group.idpath", "test-grouppath/to/whatever")
.startFromEarliestschema()
new .sinkPartitionerRoundRobinSchema()
.format(new Json().ignoreParseErrors(false .watermarkFor("ts").boundedOutOfOrderTimestamps(Duration.ofSeconds(5)))
.schema(schemalike("OrdersInKafka", LikeOption.EXCLUDING.ALL, LikeOption.INCLUDING.GENERATED)
);

The above "OrdersInFilesystem" table will be equivalent to:

tEnv.createTemporaryTable(
"OrdersInFilesystem",
new // reading from kafka table and write into filesystem table
myKafka.executeInsert(
new Connector("filesystem")
.option("path", "/path/to/whatever")
.option("format", "json")
.schema(schema)
);

Proposed Changes

TableDescriptor

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.
 */
@PublicEvolving
public abstract class TableDescriptor {

	/**
	 * 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 regsitered table path.
	 */
	public TableDescriptor like(String originalTablePath, LikeOption... likeOptions) {...}

	/**
	 * Extends some parts from the original table descriptor.
	 */
	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();
}

public class Connector extends TableDescriptor {

	private final Map<String, String> options = new HashMap<>();

	public Connector(String identifier) {
		this.options.put(CONNECTOR.key(), identifier);
	}

	public Connector option(String key, String value) {
		options.put(key, value);
		return this;
	}

	protected Map<String, String> toConnectorOptions() {
		return new HashMap<>(options);
	}
}

public class Kafka extends TableDescriptor {

	private final Map<String, String> options = new HashMap<>();

	public Kafka() {
		this.options.put(CONNECTOR.key(), "kafka");
	}

	public Kafka version(String version) {
		this.options.put(CONNECTOR.key(), "kafka-" + version);
		return this;
	}

	public Kafka topic(String topic) {
		this.options.put("topic", topic);
		return this;
	}

	public Kafka format(FormatDescriptor formatDescriptor) {
		this.options.putAll(formatDescriptor.toFormatOptions());
		return this;
	}

   ...
}

Schema

The current Rowtime class will be removed. The current Schema class will be refactored into:

schema(
new Schema()
.column("user_id", DataTypes.BIGINT())
.column("score", DataTypes.DECIMAL(10, 2))
.column("log_ts", DataTypes.TIMESTAMP(3))
.computedColumn("my_ts", "TO_TIMESTAMP(log_ts)")
.watermarkFor("ts").boundedOutOfOrderTimestamps(Duration.ofSeconds(5))
)
);

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 registering step and can use the source/sink out-of-box. For example:


Schema schema = new Schema()
.column("user_id", DataTypes.BIGINT())
.column("score", DataTypes.DECIMAL(10, 2))
.column("ts", DataTypes.TIMESTAMP(3));
Table myKafka = tEnv.from(
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(schema)
);
// reading from kafka table and write into filesystem table
myKafka.executeInsert(
new Connector("filesystem")
.option("path", "/path/to/whatever")
.option("format", "json")
.schema(schema)
);

Proposed Changes

We will discuss in detail about the new interfaces/classes in this section.


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 connectors. 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 specific descriptors. 

TableDescriptor

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.
 */
@PublicEvolving
public abstract class TableDescriptor {

	/**
	 * Specifies the table schema.
	 */
	public final TableDescriptor schema(Schema schema
Code Block
languagejava
titleSchema
/**
 * Describes a schema of a table.
 */
@PublicEvolving
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 athe watermarkpartition strategykeys forof situations with monotonously ascending timestampsthis table.
		 */
		public Schemafinal TableDescriptor ascendingTimestamps(partitionedBy(String... fieldNames) {...}

		/**
		 * SpecifiesExtends asome watermarkparts strategyfrom forthe situationsoriginal whereregsitered records are out of order, but you can place
	table path.
	 */
	public final TableDescriptor like(String originalTablePath, LikeOption... likeOptions) {...}

	/**
	 * anExtends uppersome boundparts onfrom how far the eventsoriginal are out of order. An out-of-order bound B means that
	table descriptor.
	 */
	public final TableDescriptor like(TableDescriptor originalTableDescriptor, LikeOption... likeOptions) {...}

	/**
	 * onceSpecifies the anconnector eventoptions withof timestamp T was encounteredthis table, nosubclasses eventsshould olderoverride 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

...

this method.
	 */
	protected abstract Map<String, String> connectorOptions();
}

public class Connector extends TableDescriptor {

	private final Map<String, String> options = new HashMap<>();

	public Connector(String identifier) {
		this.options.put(CONNECTOR.key(), identifier);
	}

	public Connector option(String key, String value) {
		options.put(key, value);
		return this;
	}

	protected Map<String, String> toConnectorOptions() {
		return new HashMap<>(options);
	}
}

public class Kafka extends TableDescriptor {

	private final Map<String, String> options = new HashMap<>();

	public Kafka() {
		this.options.put(CONNECTOR.key(), "kafka");
	}

	public Kafka version(String version) {
		this.options.put(CONNECTOR.key(), "kafka-" + version);
		return this;
	}

	public Kafka topic(String topic) {
		this.options.put("topic", topic);
		return this;
	}

	public Kafka format(FormatDescriptor formatDescriptor) {
		this.options.putAll(formatDescriptor.toFormatOptions());
		return this;
	}

   ...
}


Schema

The current Rowtime class will be removed. The current Schema class will be refactored into:

Code Block
languagejava
titleSchema
/**
 * Describes a schema of a table.
 */
@PublicEvolving
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) {...}
	}
}


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 give a migration plan for users who are still using TableEnvironment#connect and want to migrate to new Descriptor API. The following tables list the API changes:


Schema API Changes

Current InterfaceNew InterfaceDescription
.field("name", DataTypes.STRING()).column("name", DataTypes.STRING())
.field("name", "STRING").column("name", DataTypes.STRING())
.field("proctime", DataTypes.TIMESTAMP(3)).proctime().proctime("proctime")
.from("originalName").computedColumn("newName", "originalName")
.rowtime(new Rowtime().timestampsFromField("time").watermarksPeriodicAscending()).watermarkFor("time").ascendingTimestamps()
.rowtime(new Rowtime().timestampsFromField("time").watermarksPeriodicBounded(2)).watermarkFor("time").boundedOutOfOrderTimestamps(Duration.ofMillis(2))
rowtime.timestampsFromExtractor(TimestampExtractor).computedColumn(fieldName, expr).watermarkFor(fieldName)
rowtime.watermarksFromStrategy(WatermarkStrategy).watermarkFor(rowtimeField).as(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 InterfaceNew InterfaceDescription
.withSchema(new Schema()).schema(new Schema())
.withFormat(new Json())new Kafka().format(new Json())Moved to speicifc connector descriptor implementations.
.withPartitionKeys(Arrays.asList(a, b, c)).partitionedBy(a, b, c)
.createTemporaryTable(path)tEnv.createTemporaryTable(path, descriptor)Moved to TableEnvironment.
.inAppendMode()N/Aremoved (not needed anymore)
.inRetractMode()N/Aremoved (not needed anymore)
.inUpsertMode()N/Aremoved (not needed anymore)
.toProperties()N/Aremoved (not needed anymore)

...



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?

...