twalthr commented on a change in pull request #16142:
URL: https://github.com/apache/flink/pull/16142#discussion_r663890791
##########
File path:
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
##########
@@ -300,6 +307,246 @@ public void testKafkaTableWithMultipleTopics() throws
Exception {
topics.forEach(super::deleteTestTopic);
}
+ @Test
+ public void testKafkaSinkWithMetadataIncludeTopicOption() {
+ if (isLegacyConnector) {
+ return;
+ }
+ // we always use a different topic name for each parameterized topic,
+ // in order to make sure the topic can be created.
+ final String topic = "metadata_topic_" + format;
+ try {
+ createTestTopic(topic, 1, 1);
+
+ // ---------- Produce an event time stream into Kafka
-------------------
+ String groupId = getStandardProps().getProperty("group.id");
+ String bootstraps = getBootstrapServers();
+
+ final String createTable =
+ String.format(
+ "CREATE TABLE kafka (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\n"
+ // metadata fields are out of order on
purpose
+ // offset is ignored because it might not
be deterministic
+ + " `timestamp-type` STRING METADATA
VIRTUAL,\n"
Review comment:
simplify the test. we don't need to copy all different metadata from the
main metadata test. The schema could be simply `physical_1`, `physical_2`,
`timestamp` (to have some additional metadata), `topic`, `physical_3`
##########
File path:
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
##########
@@ -300,6 +307,246 @@ public void testKafkaTableWithMultipleTopics() throws
Exception {
topics.forEach(super::deleteTestTopic);
}
+ @Test
+ public void testKafkaSinkWithMetadataIncludeTopicOption() {
+ if (isLegacyConnector) {
+ return;
+ }
+ // we always use a different topic name for each parameterized topic,
+ // in order to make sure the topic can be created.
+ final String topic = "metadata_topic_" + format;
+ try {
+ createTestTopic(topic, 1, 1);
+
+ // ---------- Produce an event time stream into Kafka
-------------------
+ String groupId = getStandardProps().getProperty("group.id");
+ String bootstraps = getBootstrapServers();
+
+ final String createTable =
+ String.format(
+ "CREATE TABLE kafka (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\n"
+ // metadata fields are out of order on
purpose
+ // offset is ignored because it might not
be deterministic
+ + " `timestamp-type` STRING METADATA
VIRTUAL,\n"
+ + " `timestamp` TIMESTAMP(3) METADATA,\n"
+ + " `leader-epoch` INT METADATA
VIRTUAL,\n"
+ + " `headers` MAP<STRING, BYTES>
METADATA,\n"
+ + " `partition` INT METADATA VIRTUAL,\n"
+ + " `topic` STRING METADATA,\n"
+ + " `physical_3` BOOLEAN\n"
+ + ") WITH (\n"
+ + " 'connector' = 'kafka',\n"
+ + " 'topic' = '%s',\n"
+ + " 'properties.bootstrap.servers' =
'%s',\n"
+ + " 'properties.group.id' = '%s',\n"
+ + " 'scan.startup.mode' =
'earliest-offset',\n"
+ + " %s\n"
+ + ")",
+ topic, bootstraps, groupId, formatOptions());
+
+ tEnv.executeSql(createTable);
+
+ String initialValues =
+ String.format(
+ "INSERT INTO kafka\n"
+ + "VALUES\n"
+ + " ('data 1', 1, TIMESTAMP '2020-03-08
13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE'], '%s', TRUE),\n"
+ + " ('data 2', 2, TIMESTAMP '2020-03-09
13:12:11.123', CAST(NULL AS MAP<STRING, BYTES>), '%s', FALSE),\n"
+ + " ('data 3', 3, TIMESTAMP '2020-03-10
13:12:11.123', MAP['k1', X'10', 'k2', X'20'], '%s', TRUE)",
+ topic, topic, topic);
+ try {
+ tEnv.executeSql(initialValues).await();
+ fail(
+ "Unable to create the Kafka sink table with table
option 'topic' and metadata column 'topic'.");
+ } catch (Exception e) {
+ assertTrue(e instanceof ValidationException);
+ assertEquals(
+ String.format(
+ "Invalid metadata key '%s' in column 'topic'
of table 'default_catalog.default_database.kafka'. "
+ + "The %s class '%s' supports the
following metadata keys for writing:\n%s",
+ TOPIC.key,
+ DynamicTableSink.class.getSimpleName(),
+ KafkaDynamicSink.class.getName(),
+ String.join("\n", Arrays.asList(HEADERS.key,
TIMESTAMP.key))),
+ e.getMessage());
+ }
+ } finally {
+ // ------------- cleanup -------------------
+
+ deleteTestTopic(topic);
+ }
+ }
+
+ @Test
+ public void testKafkaSinkWithMetadataExcludeTopicOption() throws Exception
{
+ if (isLegacyConnector) {
+ return;
+ }
+ // we always use a different topic name for each parameterized topic,
+ // in order to make sure the topic can be created.
+ final String topic = "metadata_topic_" + format;
+ try {
+ createTestTopic(topic, 1, 1);
+
+ // ---------- Produce an event time stream into Kafka
-------------------
+ String groupId = getStandardProps().getProperty("group.id");
+ String bootstraps = getBootstrapServers();
+
+ final String createSourceTable =
+ String.format(
+ "CREATE TABLE kafka_source (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\n"
+ // metadata fields are out of order on
purpose
+ // offset is ignored because it might not
be deterministic
+ + " `timestamp-type` STRING METADATA
VIRTUAL,\n"
+ + " `timestamp` TIMESTAMP(3) METADATA,\n"
+ + " `leader-epoch` INT METADATA
VIRTUAL,\n"
+ + " `headers` MAP<STRING, BYTES>
METADATA,\n"
+ + " `partition` INT METADATA VIRTUAL,\n"
+ + " `topic` STRING METADATA VIRTUAL,\n"
+ + " `physical_3` BOOLEAN\n"
+ + ") WITH (\n"
+ + " 'connector' = 'kafka',\n"
+ + " 'topic' = '%s',\n"
+ + " 'properties.bootstrap.servers' =
'%s',\n"
+ + " 'properties.group.id' = '%s',\n"
+ + " 'scan.startup.mode' =
'earliest-offset',\n"
+ + " %s\n"
+ + ")",
+ topic, bootstraps, groupId, formatOptions());
+
+ final String createSinkTable =
+ String.format(
+ "CREATE TABLE kafka_sink (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\n"
+ // metadata fields are out of order on
purpose
+ // offset is ignored because it might not
be deterministic
+ + " `timestamp-type` STRING METADATA
VIRTUAL,\n"
+ + " `timestamp` TIMESTAMP(3) METADATA,\n"
+ + " `leader-epoch` INT METADATA
VIRTUAL,\n"
+ + " `headers` MAP<STRING, BYTES>
METADATA,\n"
+ + " `partition` INT METADATA VIRTUAL,\n"
+ + " `topic` STRING METADATA,\n"
+ + " `physical_3` BOOLEAN\n"
+ + ") WITH (\n"
+ + " 'connector' = 'kafka',\n"
+ + " 'properties.bootstrap.servers' =
'%s',\n"
+ + " 'properties.group.id' = '%s',\n"
+ + " 'scan.startup.mode' =
'earliest-offset',\n"
+ + " %s\n"
+ + ")",
+ bootstraps, groupId, formatOptions());
+
+ tEnv.executeSql(createSourceTable);
+ tEnv.executeSql(createSinkTable);
+
+ // verify whether the insertion of a null value for 'topic'
metadata column fails
+ String initialValues =
+ "INSERT INTO kafka_sink\n"
+ + "VALUES\n"
+ + " ('data 1', 1, TIMESTAMP '2020-03-08
13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE'], CAST(NULL AS VARCHAR),
TRUE),\n"
+ + " ('data 2', 2, TIMESTAMP '2020-03-09
13:12:11.123', CAST(NULL AS MAP<STRING, BYTES>), CAST(NULL AS VARCHAR),
FALSE),\n"
+ + " ('data 3', 3, TIMESTAMP '2020-03-10
13:12:11.123', MAP['k1', X'10', 'k2', X'20'], CAST(NULL AS VARCHAR), TRUE)";
+ try {
+ tEnv.executeSql(initialValues).await();
+ fail(
+ "Unable to insert Kafka sink table with null value for
'topic' metadata column.");
+ } catch (Exception e) {
+ assertTrue(
Review comment:
use `FlinkMatchers#containsCause or #containsMessage`
##########
File path:
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
##########
@@ -300,6 +307,246 @@ public void testKafkaTableWithMultipleTopics() throws
Exception {
topics.forEach(super::deleteTestTopic);
}
+ @Test
+ public void testKafkaSinkWithMetadataIncludeTopicOption() {
+ if (isLegacyConnector) {
+ return;
+ }
+ // we always use a different topic name for each parameterized topic,
+ // in order to make sure the topic can be created.
+ final String topic = "metadata_topic_" + format;
+ try {
+ createTestTopic(topic, 1, 1);
+
+ // ---------- Produce an event time stream into Kafka
-------------------
+ String groupId = getStandardProps().getProperty("group.id");
+ String bootstraps = getBootstrapServers();
+
+ final String createTable =
+ String.format(
+ "CREATE TABLE kafka (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\n"
+ // metadata fields are out of order on
purpose
+ // offset is ignored because it might not
be deterministic
+ + " `timestamp-type` STRING METADATA
VIRTUAL,\n"
+ + " `timestamp` TIMESTAMP(3) METADATA,\n"
+ + " `leader-epoch` INT METADATA
VIRTUAL,\n"
+ + " `headers` MAP<STRING, BYTES>
METADATA,\n"
+ + " `partition` INT METADATA VIRTUAL,\n"
+ + " `topic` STRING METADATA,\n"
+ + " `physical_3` BOOLEAN\n"
+ + ") WITH (\n"
+ + " 'connector' = 'kafka',\n"
+ + " 'topic' = '%s',\n"
+ + " 'properties.bootstrap.servers' =
'%s',\n"
+ + " 'properties.group.id' = '%s',\n"
+ + " 'scan.startup.mode' =
'earliest-offset',\n"
+ + " %s\n"
+ + ")",
+ topic, bootstraps, groupId, formatOptions());
+
+ tEnv.executeSql(createTable);
+
+ String initialValues =
+ String.format(
+ "INSERT INTO kafka\n"
+ + "VALUES\n"
+ + " ('data 1', 1, TIMESTAMP '2020-03-08
13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE'], '%s', TRUE),\n"
+ + " ('data 2', 2, TIMESTAMP '2020-03-09
13:12:11.123', CAST(NULL AS MAP<STRING, BYTES>), '%s', FALSE),\n"
+ + " ('data 3', 3, TIMESTAMP '2020-03-10
13:12:11.123', MAP['k1', X'10', 'k2', X'20'], '%s', TRUE)",
+ topic, topic, topic);
Review comment:
shouldn't we test at least 2 different topics here?
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaSerializationSchema.java
##########
@@ -166,7 +168,7 @@ public void setPartitions(int[] partitions) {
@Override
public String getTargetTopic(RowData element) {
- return topic == null
+ return TOPIC_UNSPECIFIED.equals(topic)
? readMetadata(element,
KafkaDynamicSink.WritableMetadata.TOPIC)
Review comment:
This comment is still not addressed. When neither option nor metadata
column are defined, `readMetadata` will return null and cause Kafka to throw an
exception. We can be smarter here and fail already during planning. In
`KafkaDynamicSink#getSinkRuntimeProvider` we have all information to create a
helpful exception if the metadata column is missing.
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaOptions.java
##########
@@ -143,8 +143,8 @@ private KafkaOptions() {}
.asList()
.noDefaultValue()
.withDescription(
- "Topic names from which the table is read. Either
'topic' or 'topic-pattern' must be set for source. "
- + "Option 'topic' is required for sink.");
+ "Topic name(s) to read data from when the table is
used as source. It also supports topic list for source by separating topic by
semicolon like 'topic-1;topic-2'. Note, only one of 'topic-pattern' and 'topic'
can be specified for sources. "
+ + "When the table is used as sink, the
topic name is the topic to write data. If the 'topic' option isn't specified
for sink table, 'topic' metadata column must be specified. If the 'topic'
option is specified for sink table, 'topic' metadata column can only be
readable. Note topic list is not supported for sinks.");
Review comment:
Maybe I missed your answer before, but why can't we allow topic list
already in this PR? We can allow users to emit to one topic of the topic list
via metadata column. I think this is even the most common case. People would
like to emit to a list of topics. The `topic` option could limit the allowed
number of metadata column values. This behavior would also be more intuitive as
the option is a "whitelist".
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java
##########
@@ -355,6 +361,22 @@ public int hashCode() {
//
--------------------------------------------------------------------------------------------
enum WritableMetadata {
+ TOPIC(
+ "topic",
+ DataTypes.STRING(),
Review comment:
I think I mentioned it before, but: declare this
`DataTypes.STRING().notNull()`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]