Airblader commented on a change in pull request #16769:
URL: https://github.com/apache/flink/pull/16769#discussion_r686286326
##########
File path:
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/DeliveryGuarantee.java
##########
@@ -17,25 +17,57 @@
package org.apache.flink.connector.base;
+import org.apache.flink.configuration.DescribedEnum;
+import org.apache.flink.configuration.description.InlineElement;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
/**
* DeliverGuarantees that can be chosen. In general your pipeline can only
offer the lowest delivery
* guarantee which is supported by your sources and sinks.
*/
-public enum DeliveryGuarantee {
+public enum DeliveryGuarantee implements DescribedEnum {
Review comment:
We should mark this `@PublicEvolving`
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java
##########
@@ -535,6 +538,15 @@ private static void autoCompleteSubject(
}
}
+ static void validateDeliveryGuarantee(ReadableConfig tableOptions) {
+ if
(tableOptions.getOptional(DELIVERY_GUARANTEE).orElse(DELIVERY_GUARANTEE.defaultValue())
+ == DeliveryGuarantee.EXACTLY_ONCE
+ &&
!tableOptions.getOptional(TRANSACTIONAL_ID_PREFIX).isPresent()) {
+ throw new ValidationException(
+ "sink.transactional-id-prefix must be specified when using
DeliveryGuarantee.EXACTLY_ONCE.");
Review comment:
nit: use `DELIVERY_GUARANTEE.key()`
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptions.java
##########
@@ -195,12 +196,6 @@
"custom class name (use
custom FlinkKafkaPartitioner subclass)"))
.build());
- public static final ConfigOption<SinkSemantic> SINK_SEMANTIC =
Review comment:
Since the values remain the same we should at least add `sink.semantic`
as a deprecated fallback key for now. The enum itself only became public during
1.14 anyway so I would expect that that isn't an issue.
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/DynamicKafkaRecordSerializationSchema.java
##########
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.table;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import
org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
+import
org.apache.flink.streaming.connectors.kafka.sink.KafkaRecordSerializationSchema;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.kafka.clients.producer.ProducerRecord;
+
+import javax.annotation.Nullable;
+
+/**
+ * SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link
+ * org.apache.flink.streaming.connectors.kafka.sink.KafkaSink}.
+ */
+class DynamicKafkaRecordSerializationSchema implements
KafkaRecordSerializationSchema<RowData> {
Review comment:
`@Internal`?
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtil.java
##########
@@ -60,14 +62,15 @@
import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER;
import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC;
import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN;
+import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX;
import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE;
import static
org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT;
import static org.apache.flink.table.factories.FactoryUtil.FORMAT;
import static
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot;
/** Utilities for {@link KafkaConnectorOptions}. */
@Internal
-class KafkaConnectorOptionsUtil {
+public class KafkaConnectorOptionsUtil {
Review comment:
Is it actually necessary to make this public?
##########
File path:
flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/DeliveryGuarantee.java
##########
@@ -17,25 +17,57 @@
package org.apache.flink.connector.base;
+import org.apache.flink.configuration.DescribedEnum;
+import org.apache.flink.configuration.description.InlineElement;
+
+import static org.apache.flink.configuration.description.TextElement.text;
+
/**
* DeliverGuarantees that can be chosen. In general your pipeline can only
offer the lowest delivery
* guarantee which is supported by your sources and sinks.
*/
-public enum DeliveryGuarantee {
+public enum DeliveryGuarantee implements DescribedEnum {
/**
* Records are only delivered exactly-once also under failover scenarios.
To build a complete
* exactly-once pipeline is required that the source and sink support
exactly-once and are
* properly configured.
*/
- EXACTLY_ONCE,
+ EXACTLY_ONCE(
+ "exactly-once",
+ text(
+ "Records are only delivered exactly-once also under
failover scenarios. To build a complete exactly-once pipeline is required that
the source and sink support exactly-once and are properly configured.")),
/**
* Records are ensured to be delivered but it may happen that the same
record is delivered
* multiple times. Usually, this guarantee is faster than the exactly-once
delivery.
*/
- AT_LEAST_ONCE,
+ AT_LEAST_ONCE(
+ "at-least-once",
+ text(
+ "Records are ensured to be delivered but it may happen
that the same record is delivered multiple times. Usually, this guarantee is
faster than the exactly-once delivery")),
Review comment:
nit: missing final period
--
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]