From 12fd4e709643e6bf1ad9ff63dceb2acfcb392092 Mon Sep 17 00:00:00 2001 From: PeterZh6 Date: Mon, 4 Nov 2024 23:40:31 +0800 Subject: [PATCH] initial commit --- .../sort-connectors/kafka/pom.xml | 192 +++++ .../inlong/sort/kafka/source/KafkaSource.java | 256 +++++++ .../sort/kafka/source/KafkaSourceBuilder.java | 591 +++++++++++++++ .../source/reader/KafkaSourceReader.java | 201 +++++ .../DynamicKafkaDeserializationSchema.java | 276 +++++++ ...DynamicKafkaRecordSerializationSchema.java | 231 ++++++ .../kafka/table/KafkaConnectorOptions.java | 359 +++++++++ .../table/KafkaConnectorOptionsUtil.java | 685 ++++++++++++++++++ .../sort/kafka/table/KafkaDynamicSink.java | 511 +++++++++++++ .../sort/kafka/table/KafkaDynamicSource.java | 684 +++++++++++++++++ .../kafka/table/KafkaDynamicTableFactory.java | 449 ++++++++++++ .../sort/kafka/table/ReducingUpsertSink.java | 104 +++ .../kafka/table/ReducingUpsertWriter.java | 190 +++++ .../sort/kafka/table/SinkBufferFlushMode.java | 79 ++ .../table/UpsertKafkaDynamicTableFactory.java | 424 +++++++++++ .../inlong/sort/kafka/util/Preconditions.java | 310 ++++++++ .../org.apache.flink.table.factories.Factory | 17 + .../sort-flink-v1.18/sort-connectors/pom.xml | 1 + 18 files changed, 5560 insertions(+) create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/util/Preconditions.java create mode 100644 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml new file mode 100644 index 00000000000..2b155ae091a --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml @@ -0,0 +1,192 @@ + + + + 4.0.0 + + org.apache.inlong + sort-connectors-v1.18 + 2.1.0-SNAPSHOT + + + sort-connector-kafka-v1.18 + jar + Apache InLong - Sort-connector-kafka + + + ${project.parent.parent.parent.parent.parent.basedir} + 3.3.0-1.19 + 3.4.0 + + + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + com.google.guava + guava + + + org.apache.inlong + sort-common + ${project.version} + + + org.apache.inlong + sort-connector-base + ${project.version} + + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.apache.flink + flink-connector-kafka + ${flink.connector.kafka.version} + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + + shade + + package + + + + org.apache.inlong:* + org.apache.kafka:* + org.apache.flink:flink-connector-kafka + org.apache.flink:flink-connector-base + + org.apache.flink:flink-shaded-guava + org.apache.httpcomponents:* + org.apache.commons:commons-lang3 + com.google.protobuf:* + com.google.guava:* + joda-time:* + com.fasterxml.jackson.core:* + com.amazonaws:* + software.amazon.ion:* + commons-logging:commons-logging + + + + + org.apache.kafka:* + + kafka/kafka-version.properties + LICENSE + + NOTICE + common/** + + + + org.apache.inlong:sort-connector-* + + org/apache/inlong/** + META-INF/services/org.apache.flink.table.factories.Factory + META-INF/services/org.apache.flink.table.factories.TableFactory + + + + + + org.apache.kafka + org.apache.inlong.sort.kafka.shaded.org.apache.kafka + + + org.apache.commons.logging + org.apache.inlong.sort.kafka.shaded.org.apache.commons.logging + + + org.apache.commons.lang3 + org.apache.inlong.sort.kafka.shaded.org.apache.commons.lang3 + + + org.apache.http + org.apache.inlong.sort.kafka.shaded.org.apache.http + + + + com.google + org.apache.inlong.sort.kafka.shaded.com.google + + + com.amazonaws + org.apache.inlong.sort.kafka.shaded.com.amazonaws + + + software.amazon.ion + org.apache.inlong.sort.kafka.shaded.software.amazon.ion + + + com.fasterxml.jackson + org.apache.inlong.sort.kafka.shaded.com.fasterxml.jackson + + + org.joda.time + org.apache.inlong.sort.kafka.shaded.org.joda.time + + + + org.apache.inlong.sort.base + org.apache.inlong.sort.kafka.shaded.org.apache.inlong.sort.base + + + org.apache.inlong.sort.configuration + org.apache.inlong.sort.kafka.shaded.org.apache.inlong.sort.configuration + + + org.apache.inlong.sort.protocol + org.apache.inlong.sort.kafka.shaded.org.apache.inlong.sort.protocol + + + org.apache.inlong.sort.schema + org.apache.inlong.sort.kafka.shaded.org.apache.inlong.sort.schema + + + org.apache.inlong.sort.util + org.apache.inlong.sort.kafka.shaded.org.apache.inlong.sort.util + + + + + + + + + + diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java new file mode 100644 index 00000000000..82878cf2e45 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java @@ -0,0 +1,256 @@ +/* + * 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.inlong.sort.kafka.source; + +import org.apache.inlong.sort.kafka.source.reader.KafkaSourceReader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.util.UserCodeClassLoader; +import org.apache.flink.util.function.SerializableSupplier; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Collection; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * The Source implementation of Kafka. Please use a {@link KafkaSourceBuilder} to construct a {@link + * KafkaSource}. The following example shows how to create a KafkaSource emitting records of + * String type. + * + *
{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+ *     .setGroupId("MyGroup")
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(new TestingKafkaRecordDeserializationSchema())
+ *     .setStartingOffsets(OffsetsInitializer.earliest())
+ *     .build();
+ * }
+ * + *

{@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator} only supports + * adding new splits and not removing splits in split discovery. + * + *

See {@link KafkaSourceBuilder} for more details on how to configure this source. + * + * @param the output type of the source. + */ +@PublicEvolving +public class KafkaSource + implements + Source, + ResultTypeQueryable { + + private static final long serialVersionUID = -8755372893283732098L; + // Users can choose only one of the following ways to specify the topics to consume from. + private final KafkaSubscriber subscriber; + // Users can specify the starting / stopping offset initializer. + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetsInitializer; + // Boundedness + private final Boundedness boundedness; + private final KafkaRecordDeserializationSchema deserializationSchema; + // The configurations. + private final Properties props; + // Client rackId callback + private final SerializableSupplier rackIdSupplier; + + KafkaSource( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetsInitializer, + @Nullable OffsetsInitializer stoppingOffsetsInitializer, + Boundedness boundedness, + KafkaRecordDeserializationSchema deserializationSchema, + Properties props, + SerializableSupplier rackIdSupplier) { + this.subscriber = subscriber; + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + this.boundedness = boundedness; + this.deserializationSchema = deserializationSchema; + this.props = props; + this.rackIdSupplier = rackIdSupplier; + } + + /** + * Get a kafkaSourceBuilder to build a {@link KafkaSource}. + * + * @return a Kafka source builder. + */ + public static KafkaSourceBuilder builder() { + return new KafkaSourceBuilder<>(); + } + + @Override + public Boundedness getBoundedness() { + return this.boundedness; + } + + @Internal + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + return createReader(readerContext, (ignore) -> { + }); + } + + @VisibleForTesting + SourceReader createReader( + SourceReaderContext readerContext, Consumer> splitFinishedHook) + throws Exception { + FutureCompletingBlockingQueue>> elementsQueue = + new FutureCompletingBlockingQueue<>(); + deserializationSchema.open( + new DeserializationSchema.InitializationContext() { + + @Override + public MetricGroup getMetricGroup() { + return readerContext.metricGroup().addGroup("deserializer"); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + }); + final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = + new KafkaSourceReaderMetrics(readerContext.metricGroup()); + + Supplier splitReaderSupplier = + () -> new KafkaPartitionSplitReader( + props, + readerContext, + kafkaSourceReaderMetrics, + Optional.ofNullable(rackIdSupplier) + .map(Supplier::get) + .orElse(null)); + KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); + + return new KafkaSourceReader<>( + elementsQueue, + new KafkaSourceFetcherManager( + elementsQueue, splitReaderSupplier::get, splitFinishedHook), + recordEmitter, + toConfiguration(props), + readerContext, + kafkaSourceReaderMetrics); + } + + @Internal + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new KafkaSourceEnumerator( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + enumContext, + boundedness); + } + + @Internal + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + KafkaSourceEnumState checkpoint) + throws IOException { + return new KafkaSourceEnumerator( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + enumContext, + boundedness, + checkpoint); + } + + @Internal + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new KafkaPartitionSplitSerializer(); + } + + @Internal + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new KafkaSourceEnumStateSerializer(); + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + // ----------- private helper methods --------------- + + private Configuration toConfiguration(Properties props) { + Configuration config = new Configuration(); + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); + return config; + } + + @VisibleForTesting + Configuration getConfiguration() { + return toConfiguration(props); + } + + @VisibleForTesting + KafkaSubscriber getKafkaSubscriber() { + return subscriber; + } + + @VisibleForTesting + OffsetsInitializer getStoppingOffsetsInitializer() { + return stoppingOffsetsInitializer; + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java new file mode 100644 index 00000000000..2c70b289c4b --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java @@ -0,0 +1,591 @@ +/* + * 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.inlong.sort.kafka.source; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.function.SerializableSupplier; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * The builder class for {@link KafkaSource} to make it easier for the users to construct a {@link + * KafkaSource}. + * + *

The following example shows the minimum setup to create a KafkaSource that reads the String + * values from a Kafka topic. + * + *

{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
+ *     .build();
+ * }
+ * + *

The bootstrap servers, topics/partitions to consume, and the record deserializer are required + * fields that must be set. + * + *

To specify the starting offsets of the KafkaSource, one can call {@link + * #setStartingOffsets(OffsetsInitializer)}. + * + *

By default the KafkaSource runs in an {@link Boundedness#CONTINUOUS_UNBOUNDED} mode and never + * stops until the Flink job is canceled or fails. To let the KafkaSource run as {@link + * Boundedness#CONTINUOUS_UNBOUNDED} yet stop at some given offsets, one can call {@link + * #setUnbounded(OffsetsInitializer)}. For example the following KafkaSource stops after it consumes + * up to the latest partition offsets at the point when the Flink job started. + * + *

{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
+ *     .setUnbounded(OffsetsInitializer.latest())
+ *     .setRackId(() -> MY_RACK_ID)
+ *     .build();
+ * }
+ * + *

Check the Java docs of each individual methods to learn more about the settings to build a + * KafkaSource. + */ +@PublicEvolving +public class KafkaSourceBuilder { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceBuilder.class); + private static final String[] REQUIRED_CONFIGS = {ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}; + // The subscriber specifies the partitions to subscribe to. + private KafkaSubscriber subscriber; + // Users can specify the starting / stopping offset initializer. + private OffsetsInitializer startingOffsetsInitializer; + private OffsetsInitializer stoppingOffsetsInitializer; + // Boundedness + private Boundedness boundedness; + private KafkaRecordDeserializationSchema deserializationSchema; + // The configurations. + protected Properties props; + // Client rackId supplier + private SerializableSupplier rackIdSupplier; + + KafkaSourceBuilder() { + this.subscriber = null; + this.startingOffsetsInitializer = OffsetsInitializer.earliest(); + this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.deserializationSchema = null; + this.props = new Properties(); + this.rackIdSupplier = null; + } + + /** + * Sets the bootstrap servers for the KafkaConsumer of the KafkaSource. + * + * @param bootstrapServers the bootstrap servers of the Kafka cluster. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setBootstrapServers(String bootstrapServers) { + return setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } + + /** + * Sets the consumer group id of the KafkaSource. + * + * @param groupId the group id of the KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setGroupId(String groupId) { + return setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); + } + + /** + * Set a list of topics the KafkaSource should consume from. All the topics in the list should + * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of + * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead. + * + * @param topics the list of topics to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection) + */ + public KafkaSourceBuilder setTopics(List topics) { + ensureSubscriberIsNull("topics"); + subscriber = KafkaSubscriber.getTopicListSubscriber(topics); + return this; + } + + /** + * Set a list of topics the KafkaSource should consume from. All the topics in the list should + * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of + * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead. + * + * @param topics the list of topics to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection) + */ + public KafkaSourceBuilder setTopics(String... topics) { + return setTopics(Arrays.asList(topics)); + } + + /** + * Set a topic pattern to consume from use the java {@link Pattern}. + * + * @param topicPattern the pattern of the topic name to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Pattern) + */ + public KafkaSourceBuilder setTopicPattern(Pattern topicPattern) { + ensureSubscriberIsNull("topic pattern"); + subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern); + return this; + } + + /** + * Set a set of partitions to consume from. + * + * @param partitions the set of partitions to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#assign(Collection) + */ + public KafkaSourceBuilder setPartitions(Set partitions) { + ensureSubscriberIsNull("partitions"); + subscriber = KafkaSubscriber.getPartitionSetSubscriber(partitions); + return this; + } + + /** + * Set a custom Kafka subscriber to use to discover new splits. + * + * @param kafkaSubscriber the {@link KafkaSubscriber} to use for split discovery. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setKafkaSubscriber(KafkaSubscriber kafkaSubscriber) { + ensureSubscriberIsNull("custom"); + this.subscriber = checkNotNull(kafkaSubscriber); + return this; + } + + /** + * Specify from which offsets the KafkaSource should start consuming from by providing an {@link + * OffsetsInitializer}. + * + *

The following {@link OffsetsInitializer}s are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

+ * + * @param startingOffsetsInitializer the {@link OffsetsInitializer} setting the starting offsets + * for the Source. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setStartingOffsets( + OffsetsInitializer startingOffsetsInitializer) { + this.startingOffsetsInitializer = startingOffsetsInitializer; + return this; + } + + /** + * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus + * never stops until the Flink job fails or is canceled. To let the KafkaSource run as a + * streaming source but still stop at some point, one can set an {@link OffsetsInitializer} to + * specify the stopping offsets for each partition. When all the partitions have reached their + * stopping offsets, the KafkaSource will then exit. + * + *

This method is different from {@link #setBounded(OffsetsInitializer)} in that after + * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will + * still return {@link Boundedness#CONTINUOUS_UNBOUNDED} even though it will stop at the + * stopping offsets specified by the stopping offsets {@link OffsetsInitializer}. + * + *

The following {@link OffsetsInitializer} are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

+ * + * @param stoppingOffsetsInitializer The {@link OffsetsInitializer} to specify the stopping + * offset. + * @return this KafkaSourceBuilder. + * @see #setBounded(OffsetsInitializer) + */ + public KafkaSourceBuilder setUnbounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus + * never stops until the Flink job fails or is canceled. To let the KafkaSource run as {@link + * Boundedness#BOUNDED} and stop at some point, one can set an {@link OffsetsInitializer} to + * specify the stopping offsets for each partition. When all the partitions have reached their + * stopping offsets, the KafkaSource will then exit. + * + *

This method is different from {@link #setUnbounded(OffsetsInitializer)} in that after + * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will + * return {@link Boundedness#BOUNDED} instead of {@link Boundedness#CONTINUOUS_UNBOUNDED}. + * + *

The following {@link OffsetsInitializer} are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

    + *
  • {@link OffsetsInitializer#latest()} - stop at the latest offsets of the partitions when + * the KafkaSource starts to run. + *
  • {@link OffsetsInitializer#committedOffsets()} - stops at the committed offsets of the + * consumer group. + *
  • {@link OffsetsInitializer#offsets(Map)} - stops at the specified offsets for each + * partition. + *
  • {@link OffsetsInitializer#timestamp(long)} - stops at the specified timestamp for each + * partition. The guarantee of setting the stopping timestamp is that no Kafka records + * whose {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater + * than the given stopping timestamp will be consumed. However, it is possible that some + * records whose timestamp is smaller than the specified stopping timestamp are not + * consumed. + *
+ * + * @param stoppingOffsetsInitializer the {@link OffsetsInitializer} to specify the stopping + * offsets. + * @return this KafkaSourceBuilder. + * @see #setUnbounded(OffsetsInitializer) + */ + public KafkaSourceBuilder setBounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.BOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource. + * + * @param recordDeserializer the deserializer for Kafka {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord}. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setDeserializer( + KafkaRecordDeserializationSchema recordDeserializer) { + this.deserializationSchema = recordDeserializer; + return this; + } + + /** + * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource. The given + * {@link DeserializationSchema} will be used to deserialize the value of ConsumerRecord. The + * other information (e.g. key) in a ConsumerRecord will be ignored. + * + * @param deserializationSchema the {@link DeserializationSchema} to use for deserialization. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setValueOnlyDeserializer( + DeserializationSchema deserializationSchema) { + this.deserializationSchema = + KafkaRecordDeserializationSchema.valueOnly(deserializationSchema); + return this; + } + + /** + * Sets the client id prefix of this KafkaSource. + * + * @param prefix the client id prefix to use for this KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setClientIdPrefix(String prefix) { + return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); + } + + /** + * Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader. + * + * @param rackIdCallback callback to provide Kafka consumer client.rack + * @return this KafkaSourceBuilder + */ + public KafkaSourceBuilder setRackIdSupplier(SerializableSupplier rackIdCallback) { + this.rackIdSupplier = rackIdCallback; + return this; + } + + /** + * Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found + * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. + * + *

Note that the following keys will be overridden by the builder when the KafkaSource is + * created. + * + *

    + *
  • auto.offset.reset.strategy is overridden by {@link + * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by + * default {@link OffsetsInitializer#earliest()}. + *
  • partition.discovery.interval.ms is overridden to -1 when {@link + * #setBounded(OffsetsInitializer)} has been invoked. + *
+ * + * @param key the key of the property. + * @param value the value of the property. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setProperty(String key, String value) { + props.setProperty(key, value); + return this; + } + + /** + * Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found + * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. + * + *

Note that the following keys will be overridden by the builder when the KafkaSource is + * created. + * + *

    + *
  • auto.offset.reset.strategy is overridden by {@link + * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by + * default {@link OffsetsInitializer#earliest()}. + *
  • partition.discovery.interval.ms is overridden to -1 when {@link + * #setBounded(OffsetsInitializer)} has been invoked. + *
  • client.id is overridden to the "client.id.prefix-RANDOM_LONG", or + * "group.id-RANDOM_LONG" if the client id prefix is not set. + *
+ * + * @param props the properties to set for the KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setProperties(Properties props) { + this.props.putAll(props); + return this; + } + + /** + * Build the {@link KafkaSource}. + * + * @return a KafkaSource with the settings made for this builder. + */ + public KafkaSource build() { + sanityCheck(); + parseAndSetRequiredProperties(); + return new KafkaSource<>( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + boundedness, + deserializationSchema, + props, + rackIdSupplier); + } + + // ------------- private helpers -------------- + + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (subscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, subscriber.getClass().getSimpleName())); + } + } + + private void parseAndSetRequiredProperties() { + maybeOverride( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + false); + maybeOverride( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + false); + if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + LOG.warn( + "Offset commit on checkpoint is disabled because {} is not specified", + ConsumerConfig.GROUP_ID_CONFIG); + maybeOverride(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false", false); + } + maybeOverride(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false", false); + maybeOverride( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase(), + true); + + // If the source is bounded, do not run periodic partition discovery. + maybeOverride( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + + // If the client id prefix is not set, reuse the consumer group id as the client id prefix, + // or generate a random string if consumer group id is not specified. + maybeOverride( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) + ? props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) + : "KafkaSource-" + new Random().nextLong(), + false); + } + + private boolean maybeOverride(String key, String value, boolean override) { + boolean overridden = false; + String userValue = props.getProperty(key); + if (userValue != null) { + if (override) { + LOG.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + props.setProperty(key, value); + overridden = true; + } + } else { + props.setProperty(key, value); + } + return overridden; + } + + private void sanityCheck() { + // Check required configs. + for (String requiredConfig : REQUIRED_CONFIGS) { + checkNotNull( + props.getProperty(requiredConfig), + String.format("Property %s is required but not provided", requiredConfig)); + } + // Check required settings. + checkNotNull( + subscriber, + "No subscribe mode is specified, " + + "should be one of topics, topic pattern and partition set."); + checkNotNull(deserializationSchema, "Deserialization schema is required but not provided."); + // Check consumer group ID + checkState( + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(), + String.format( + "Property %s is required when offset commit is enabled", + ConsumerConfig.GROUP_ID_CONFIG)); + // Check offsets initializers + if (startingOffsetsInitializer instanceof OffsetsInitializerValidator) { + ((OffsetsInitializerValidator) startingOffsetsInitializer).validate(props); + } + if (stoppingOffsetsInitializer instanceof OffsetsInitializerValidator) { + ((OffsetsInitializerValidator) stoppingOffsetsInitializer).validate(props); + } + if (props.containsKey(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)); + } + if (props.containsKey(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)) { + checkDeserializer(props.getProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)); + } + } + + private void checkDeserializer(String deserializer) { + try { + Class deserClass = Class.forName(deserializer); + if (!Deserializer.class.isAssignableFrom(deserClass)) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s is not a subclass of %s", + deserializer, Deserializer.class.getName())); + } + + // Get the generic type information + Type[] interfaces = deserClass.getGenericInterfaces(); + for (Type iface : interfaces) { + if (iface instanceof ParameterizedType) { + ParameterizedType parameterizedType = (ParameterizedType) iface; + Type rawType = parameterizedType.getRawType(); + + // Check if it's Deserializer + if (rawType == Deserializer.class) { + Type[] typeArguments = parameterizedType.getActualTypeArguments(); + if (typeArguments.length != 1 || typeArguments[0] != byte[].class) { + throw new IllegalArgumentException( + String.format( + "Deserializer class %s does not deserialize byte[]", + deserializer)); + } + } + } + } + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException( + String.format("Deserializer class %s not found", deserializer), e); + } + } + + private boolean offsetCommitEnabledManually() { + boolean autoCommit = + props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + && Boolean.parseBoolean( + props.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + boolean commitOnCheckpoint = + props.containsKey(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key()) + && Boolean.parseBoolean( + props.getProperty( + KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key())); + return autoCommit || commitOnCheckpoint; + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java new file mode 100644 index 00000000000..230a3df8165 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java @@ -0,0 +1,201 @@ +/* + * 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.inlong.sort.kafka.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitState; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** The source reader for Kafka partitions. */ +@Internal +public class KafkaSourceReader + extends + SingleThreadMultiplexSourceReaderBase, T, KafkaPartitionSplit, KafkaPartitionSplitState> { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceReader.class); + // These maps need to be concurrent because it will be accessed by both the main thread + // and the split fetcher thread in the callback. + private final SortedMap> offsetsToCommit; + private final ConcurrentMap offsetsOfFinishedSplits; + private final KafkaSourceReaderMetrics kafkaSourceReaderMetrics; + private final boolean commitOffsetsOnCheckpoint; + + public KafkaSourceReader( + FutureCompletingBlockingQueue>> elementsQueue, + KafkaSourceFetcherManager kafkaSourceFetcherManager, + RecordEmitter, T, KafkaPartitionSplitState> recordEmitter, + Configuration config, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics) { + super(elementsQueue, kafkaSourceFetcherManager, recordEmitter, config, context); + this.offsetsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.offsetsOfFinishedSplits = new ConcurrentHashMap<>(); + this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics; + this.commitOffsetsOnCheckpoint = + config.get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT); + if (!commitOffsetsOnCheckpoint) { + LOG.warn( + "Offset commit on checkpoint is disabled. " + + "Consuming offset will not be reported back to Kafka cluster."); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + finishedSplitIds.forEach( + (ignored, splitState) -> { + if (splitState.getCurrentOffset() >= 0) { + offsetsOfFinishedSplits.put( + splitState.getTopicPartition(), + new OffsetAndMetadata(splitState.getCurrentOffset())); + } + }); + } + + @Override + public List snapshotState(long checkpointId) { + List splits = super.snapshotState(checkpointId); + if (!commitOffsetsOnCheckpoint) { + return splits; + } + + if (splits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) { + offsetsToCommit.put(checkpointId, Collections.emptyMap()); + } else { + Map offsetsMap = + offsetsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>()); + // Put the offsets of the active splits. + for (KafkaPartitionSplit split : splits) { + // If the checkpoint is triggered before the partition starting offsets + // is retrieved, do not commit the offsets for those partitions. + if (split.getStartingOffset() >= 0) { + offsetsMap.put( + split.getTopicPartition(), + new OffsetAndMetadata(split.getStartingOffset())); + } + } + // Put offsets of all the finished splits. + offsetsMap.putAll(offsetsOfFinishedSplits); + } + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing offsets for checkpoint {}", checkpointId); + if (!commitOffsetsOnCheckpoint) { + return; + } + + Map committedPartitions = + offsetsToCommit.get(checkpointId); + if (committedPartitions == null) { + LOG.debug("Offsets for checkpoint {} have already been committed.", checkpointId); + return; + } + + if (committedPartitions.isEmpty()) { + LOG.debug("There are no offsets to commit for checkpoint {}.", checkpointId); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + return; + } + + ((KafkaSourceFetcherManager) splitFetcherManager) + .commitOffsets( + committedPartitions, + (ignored, e) -> { + // The offset commit here is needed by the external monitoring. It won't + // break Flink job's correctness if we fail to commit the offset here. + if (e != null) { + kafkaSourceReaderMetrics.recordFailedCommit(); + LOG.warn( + "Failed to commit consumer offsets for checkpoint {}", + checkpointId, + e); + } else { + LOG.debug( + "Successfully committed offsets for checkpoint {}", + checkpointId); + kafkaSourceReaderMetrics.recordSucceededCommit(); + // If the finished topic partition has been committed, we remove it + // from the offsets of the finished splits map. + committedPartitions.forEach( + (tp, offset) -> kafkaSourceReaderMetrics.recordCommittedOffset( + tp, offset.offset())); + offsetsOfFinishedSplits + .entrySet() + .removeIf( + entry -> committedPartitions.containsKey( + entry.getKey())); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + } + }); + } + + private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { + while (!offsetsToCommit.isEmpty() && offsetsToCommit.firstKey() <= checkpointId) { + offsetsToCommit.remove(offsetsToCommit.firstKey()); + } + } + + @Override + protected KafkaPartitionSplitState initializedState(KafkaPartitionSplit split) { + return new KafkaPartitionSplitState(split); + } + + @Override + protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitState splitState) { + return splitState.toKafkaPartitionSplit(); + } + + // ------------------------ + + @VisibleForTesting + SortedMap> getOffsetsToCommit() { + return offsetsToCommit; + } + + @VisibleForTesting + int getNumAliveFetchers() { + return splitFetcherManager.getNumAliveFetchers(); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java new file mode 100644 index 00000000000..8f8353b6942 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java @@ -0,0 +1,276 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** A specific {@link KafkaSerializationSchema} for {@link KafkaDynamicSource}. */ +@Internal +class DynamicKafkaDeserializationSchema implements KafkaDeserializationSchema { + + private static final long serialVersionUID = 1L; + + private final @Nullable DeserializationSchema keyDeserialization; + + private final DeserializationSchema valueDeserialization; + + private final boolean hasMetadata; + + private final BufferingCollector keyCollector; + + private final OutputProjectionCollector outputCollector; + + private final TypeInformation producedTypeInfo; + + private final boolean upsertMode; + + DynamicKafkaDeserializationSchema( + int physicalArity, + @Nullable DeserializationSchema keyDeserialization, + int[] keyProjection, + DeserializationSchema valueDeserialization, + int[] valueProjection, + boolean hasMetadata, + MetadataConverter[] metadataConverters, + TypeInformation producedTypeInfo, + boolean upsertMode) { + if (upsertMode) { + Preconditions.checkArgument( + keyDeserialization != null && keyProjection.length > 0, + "Key must be set in upsert mode for deserialization schema."); + } + this.keyDeserialization = keyDeserialization; + this.valueDeserialization = valueDeserialization; + this.hasMetadata = hasMetadata; + this.keyCollector = new BufferingCollector(); + this.outputCollector = + new OutputProjectionCollector( + physicalArity, + keyProjection, + valueProjection, + metadataConverters, + upsertMode); + this.producedTypeInfo = producedTypeInfo; + this.upsertMode = upsertMode; + } + + @Override + public void open(DeserializationSchema.InitializationContext context) throws Exception { + if (keyDeserialization != null) { + keyDeserialization.open(context); + } + valueDeserialization.open(context); + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public RowData deserialize(ConsumerRecord record) throws Exception { + throw new IllegalStateException("A collector is required for deserializing."); + } + + @Override + public void deserialize(ConsumerRecord record, Collector collector) + throws Exception { + // shortcut in case no output projection is required, + // also not for a cartesian product with the keys + if (keyDeserialization == null && !hasMetadata) { + valueDeserialization.deserialize(record.value(), collector); + return; + } + + // buffer key(s) + if (keyDeserialization != null) { + keyDeserialization.deserialize(record.key(), keyCollector); + } + + // project output while emitting values + outputCollector.inputRecord = record; + outputCollector.physicalKeyRows = keyCollector.buffer; + outputCollector.outputCollector = collector; + if (record.value() == null && upsertMode) { + // collect tombstone messages in upsert mode by hand + outputCollector.collect(null); + } else { + valueDeserialization.deserialize(record.value(), outputCollector); + } + keyCollector.buffer.clear(); + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + // -------------------------------------------------------------------------------------------- + + interface MetadataConverter extends Serializable { + + Object read(ConsumerRecord record); + } + + // -------------------------------------------------------------------------------------------- + + private static final class BufferingCollector implements Collector, Serializable { + + private static final long serialVersionUID = 1L; + + private final List buffer = new ArrayList<>(); + + @Override + public void collect(RowData record) { + buffer.add(record); + } + + @Override + public void close() { + // nothing to do + } + } + + // -------------------------------------------------------------------------------------------- + + /** + * Emits a row with key, value, and metadata fields. + * + *

The collector is able to handle the following kinds of keys: + * + *

    + *
  • No key is used. + *
  • A key is used. + *
  • The deserialization schema emits multiple keys. + *
  • Keys and values have overlapping fields. + *
  • Keys are used and value is null. + *
+ */ + private static final class OutputProjectionCollector + implements + Collector, + Serializable { + + private static final long serialVersionUID = 1L; + + private final int physicalArity; + + private final int[] keyProjection; + + private final int[] valueProjection; + + private final MetadataConverter[] metadataConverters; + + private final boolean upsertMode; + + private transient ConsumerRecord inputRecord; + + private transient List physicalKeyRows; + + private transient Collector outputCollector; + + OutputProjectionCollector( + int physicalArity, + int[] keyProjection, + int[] valueProjection, + MetadataConverter[] metadataConverters, + boolean upsertMode) { + this.physicalArity = physicalArity; + this.keyProjection = keyProjection; + this.valueProjection = valueProjection; + this.metadataConverters = metadataConverters; + this.upsertMode = upsertMode; + } + + @Override + public void collect(RowData physicalValueRow) { + // no key defined + if (keyProjection.length == 0) { + emitRow(null, (GenericRowData) physicalValueRow); + return; + } + + // otherwise emit a value for each key + for (RowData physicalKeyRow : physicalKeyRows) { + emitRow((GenericRowData) physicalKeyRow, (GenericRowData) physicalValueRow); + } + } + + @Override + public void close() { + // nothing to do + } + + private void emitRow( + @Nullable GenericRowData physicalKeyRow, + @Nullable GenericRowData physicalValueRow) { + final RowKind rowKind; + if (physicalValueRow == null) { + if (upsertMode) { + rowKind = RowKind.DELETE; + } else { + throw new DeserializationException( + "Invalid null value received in non-upsert mode. Could not to set row kind for output record."); + } + } else { + rowKind = physicalValueRow.getRowKind(); + } + + final int metadataArity = metadataConverters.length; + final GenericRowData producedRow = + new GenericRowData(rowKind, physicalArity + metadataArity); + + for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) { + assert physicalKeyRow != null; + producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos)); + } + + if (physicalValueRow != null) { + for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) { + producedRow.setField( + valueProjection[valuePos], physicalValueRow.getField(valuePos)); + } + } + + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + physicalArity + metadataPos, + metadataConverters[metadataPos].read(inputRecord)); + } + + outputCollector.collect(producedRow); + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java new file mode 100644 index 00000000000..0eded1dbdf1 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -0,0 +1,231 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaSink; +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; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. */ +@Internal +class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { + + private final Set topics; + private final Pattern topicPattern; + private final KafkaPartitioner partitioner; + @Nullable + private final SerializationSchema keySerialization; + private final SerializationSchema valueSerialization; + private final RowData.FieldGetter[] keyFieldGetters; + private final RowData.FieldGetter[] valueFieldGetters; + private final boolean hasMetadata; + private final int[] metadataPositions; + private final boolean upsertMode; + private final Map topicPatternMatches; + + DynamicKafkaRecordSerializationSchema( + @Nullable List topics, + @Nullable Pattern topicPattern, + @Nullable KafkaPartitioner partitioner, + @Nullable SerializationSchema keySerialization, + SerializationSchema valueSerialization, + RowData.FieldGetter[] keyFieldGetters, + RowData.FieldGetter[] valueFieldGetters, + boolean hasMetadata, + int[] metadataPositions, + boolean upsertMode) { + if (upsertMode) { + Preconditions.checkArgument( + keySerialization != null && keyFieldGetters.length > 0, + "Key must be set in upsert mode for serialization schema."); + } + Preconditions.checkArgument( + (topics != null && topicPattern == null && topics.size() > 0) + || (topics == null && topicPattern != null), + "Either Topic or Topic Pattern must be set."); + if (topics != null) { + this.topics = new HashSet<>(topics); + } else { + this.topics = null; + } + this.topicPattern = topicPattern; + this.partitioner = partitioner; + this.keySerialization = keySerialization; + this.valueSerialization = checkNotNull(valueSerialization); + this.keyFieldGetters = keyFieldGetters; + this.valueFieldGetters = valueFieldGetters; + this.hasMetadata = hasMetadata; + this.metadataPositions = metadataPositions; + this.upsertMode = upsertMode; + // Cache results of topic pattern matches to avoid re-evaluating the pattern for each record + this.topicPatternMatches = new HashMap<>(); + } + + @Override + public ProducerRecord serialize( + RowData consumedRow, KafkaSinkContext context, Long timestamp) { + // shortcut in case no input projection is required + if (keySerialization == null && !hasMetadata) { + final byte[] valueSerialized = valueSerialization.serialize(consumedRow); + final String targetTopic = getTargetTopic(consumedRow); + return new ProducerRecord<>( + targetTopic, + extractPartition( + consumedRow, + targetTopic, + null, + valueSerialized, + context.getPartitionsForTopic(targetTopic)), + null, + valueSerialized); + } + final byte[] keySerialized; + if (keySerialization == null) { + keySerialized = null; + } else { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + keySerialized = keySerialization.serialize(keyRow); + } + + final byte[] valueSerialized; + final RowKind kind = consumedRow.getRowKind(); + if (upsertMode) { + if (kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE) { + // transform the message as the tombstone message + valueSerialized = null; + } else { + // make the message to be INSERT to be compliant with the INSERT-ONLY format + final RowData valueRow = + DynamicKafkaRecordSerializationSchema.createProjectedRow( + consumedRow, kind, valueFieldGetters); + valueRow.setRowKind(RowKind.INSERT); + valueSerialized = valueSerialization.serialize(valueRow); + } + } else { + final RowData valueRow = + DynamicKafkaRecordSerializationSchema.createProjectedRow( + consumedRow, kind, valueFieldGetters); + valueSerialized = valueSerialization.serialize(valueRow); + } + final String targetTopic = getTargetTopic(consumedRow); + return new ProducerRecord<>( + targetTopic, + extractPartition( + consumedRow, + targetTopic, + keySerialized, + valueSerialized, + context.getPartitionsForTopic(targetTopic)), + readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.TIMESTAMP), + keySerialized, + valueSerialized, + readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.HEADERS)); + } + + @Override + public void open( + SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext) + throws Exception { + if (keySerialization != null) { + keySerialization.open(context); + } + if (partitioner != null) { + partitioner.open( + sinkContext.getParallelInstanceId(), + sinkContext.getNumberOfParallelInstances()); + } + valueSerialization.open(context); + } + + private String getTargetTopic(RowData element) { + if (topics != null && topics.size() == 1) { + // If topics is a singleton list, we only return the provided topic. + return topics.stream().findFirst().get(); + } + final String targetTopic = readMetadata(element, KafkaDynamicSink.WritableMetadata.TOPIC); + if (targetTopic == null) { + throw new IllegalArgumentException( + "The topic of the sink record is not valid. Expected a single topic but no topic is set."); + } else if (topics != null && !topics.contains(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to be in: %s but was: %s", + topics, targetTopic)); + } else if (topicPattern != null && !cachedTopicPatternMatch(targetTopic)) { + throw new IllegalArgumentException( + String.format( + "The topic of the sink record is not valid. Expected topic to match: %s but was: %s", + topicPattern, targetTopic)); + } + return targetTopic; + } + + private boolean cachedTopicPatternMatch(String topic) { + return topicPatternMatches.computeIfAbsent(topic, t -> topicPattern.matcher(t).matches()); + } + + private Integer extractPartition( + RowData consumedRow, + String targetTopic, + @Nullable byte[] keySerialized, + byte[] valueSerialized, + int[] partitions) { + if (partitioner != null) { + return partitioner.partition( + consumedRow, keySerialized, valueSerialized, targetTopic, partitions); + } + return null; + } + + static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, KafkaDynamicSink.WritableMetadata metadata) { + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java new file mode 100644 index 00000000000..a0c731a7b88 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java @@ -0,0 +1,359 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.DescribedEnum; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.InlineElement; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.table.factories.FactoryUtil; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX; + +/** Options for the Kafka connector. */ +@PublicEvolving +public class KafkaConnectorOptions { + + // -------------------------------------------------------------------------------------------- + // Format options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption KEY_FORMAT = + ConfigOptions.key("key" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for encoding key data. " + + "The identifier is used to discover a suitable format factory."); + + public static final ConfigOption VALUE_FORMAT = + ConfigOptions.key("value" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for encoding value data. " + + "The identifier is used to discover a suitable format factory."); + + public static final ConfigOption> KEY_FIELDS = + ConfigOptions.key("key.fields") + .stringType() + .asList() + .defaultValues() + .withDescription( + "Defines an explicit list of physical columns from the table schema " + + "that configure the data type for the key format. By default, this list is " + + "empty and thus a key is undefined."); + + public static final ConfigOption VALUE_FIELDS_INCLUDE = + ConfigOptions.key("value.fields-include") + .enumType(ValueFieldsStrategy.class) + .defaultValue(ValueFieldsStrategy.ALL) + .withDescription( + String.format( + "Defines a strategy how to deal with key columns in the data type " + + "of the value format. By default, '%s' physical columns of the table schema " + + "will be included in the value format which means that the key columns " + + "appear in the data type for both the key and value format.", + ValueFieldsStrategy.ALL)); + + public static final ConfigOption KEY_FIELDS_PREFIX = + ConfigOptions.key("key.fields-prefix") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Defines a custom prefix for all fields of the key format to avoid " + + "name clashes with fields of the value format. " + + "By default, the prefix is empty.") + .linebreak() + .text( + String.format( + "If a custom prefix is defined, both the table schema and '%s' will work with prefixed names.", + KEY_FIELDS.key())) + .linebreak() + .text( + "When constructing the data type of the key format, the prefix " + + "will be removed and the non-prefixed names will be used within the key format.") + .linebreak() + .text( + String.format( + "Please note that this option requires that '%s' must be '%s'.", + VALUE_FIELDS_INCLUDE.key(), + ValueFieldsStrategy.EXCEPT_KEY)) + .build()); + + public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; + + // -------------------------------------------------------------------------------------------- + // Kafka specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption> TOPIC = + ConfigOptions.key("topic") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + "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. It also supports topic list for sinks. The provided topic-list is treated as a allow list of valid values for the `topic` metadata column. If a list is provided, for sink table, 'topic' metadata column is writable and must be specified."); + + public static final ConfigOption TOPIC_PATTERN = + ConfigOptions.key("topic-pattern") + .stringType() + .noDefaultValue() + .withDescription( + "Optional topic pattern from which the table is read for source, or topic pattern that must match the provided `topic` metadata column for sink. Either 'topic' or 'topic-pattern' must be set."); + + public static final ConfigOption PROPS_BOOTSTRAP_SERVERS = + ConfigOptions.key("properties.bootstrap.servers") + .stringType() + .noDefaultValue() + .withDescription("Required Kafka server connection string"); + + public static final ConfigOption PROPS_GROUP_ID = + ConfigOptions.key("properties.group.id") + .stringType() + .noDefaultValue() + .withDescription( + "Required consumer group in Kafka consumer, no need for Kafka producer"); + + // -------------------------------------------------------------------------------------------- + // Scan specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SCAN_STARTUP_MODE = + ConfigOptions.key("scan.startup.mode") + .enumType(ScanStartupMode.class) + .defaultValue(ScanStartupMode.GROUP_OFFSETS) + .withDescription("Startup mode for Kafka consumer."); + + public static final ConfigOption SCAN_BOUNDED_MODE = + ConfigOptions.key("scan.bounded.mode") + .enumType(ScanBoundedMode.class) + .defaultValue(ScanBoundedMode.UNBOUNDED) + .withDescription("Bounded mode for Kafka consumer."); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSETS = + ConfigOptions.key("scan.startup.specific-offsets") + .stringType() + .noDefaultValue() + .withDescription( + "Optional offsets used in case of \"specific-offsets\" startup mode"); + + public static final ConfigOption SCAN_BOUNDED_SPECIFIC_OFFSETS = + ConfigOptions.key("scan.bounded.specific-offsets") + .stringType() + .noDefaultValue() + .withDescription( + "Optional offsets used in case of \"specific-offsets\" bounded mode"); + + public static final ConfigOption SCAN_STARTUP_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.startup.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"timestamp\" startup mode"); + + public static final ConfigOption SCAN_BOUNDED_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.bounded.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"timestamp\" bounded mode"); + + public static final ConfigOption SCAN_TOPIC_PARTITION_DISCOVERY = + ConfigOptions.key("scan.topic-partition-discovery.interval") + .durationType() + .defaultValue(Duration.ofMinutes(5)) + .withDescription( + "Optional interval for consumer to discover dynamically created Kafka partitions periodically." + + "The value 0 disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); + + // -------------------------------------------------------------------------------------------- + // Sink specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SINK_PARTITIONER = + ConfigOptions.key("sink.partitioner") + .stringType() + .defaultValue("default") + .withDescription( + Description.builder() + .text( + "Optional output partitioning from Flink's partitions into Kafka's partitions. Valid enumerations are") + .list( + text( + "'default' (use kafka default partitioner to partition records)"), + text( + "'fixed' (each Flink partition ends up in at most one Kafka partition)"), + text( + "'round-robin' (a Flink partition is distributed to Kafka partitions round-robin when 'key.fields' is not specified)"), + text( + "custom class name (use custom FlinkKafkaPartitioner subclass)")) + .build()); + + // Disable this feature by default + public static final ConfigOption SINK_BUFFER_FLUSH_MAX_ROWS = + ConfigOptions.key("sink.buffer-flush.max-rows") + .intType() + .defaultValue(0) + .withDescription( + Description.builder() + .text( + "The max size of buffered records before flushing. " + + "When the sink receives many updates on the same key, " + + "the buffer will retain the last records of the same key. " + + "This can help to reduce data shuffling and avoid possible tombstone messages to the Kafka topic.") + .linebreak() + .text("Can be set to '0' to disable it.") + .linebreak() + .text( + "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + + "must be set to be greater than zero to enable sink buffer flushing.") + .build()); + + // Disable this feature by default + public static final ConfigOption SINK_BUFFER_FLUSH_INTERVAL = + ConfigOptions.key("sink.buffer-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(0)) + .withDescription( + Description.builder() + .text( + "The flush interval millis. Over this time, asynchronous threads " + + "will flush data. When the sink receives many updates on the same key, " + + "the buffer will retain the last record of the same key.") + .linebreak() + .text("Can be set to '0' to disable it.") + .linebreak() + .text( + "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + + "must be set to be greater than zero to enable sink buffer flushing.") + .build()); + + public static final ConfigOption DELIVERY_GUARANTEE = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) + .withDescription("Optional delivery guarantee when committing."); + + public static final ConfigOption TRANSACTIONAL_ID_PREFIX = + ConfigOptions.key("sink.transactional-id-prefix") + .stringType() + .noDefaultValue() + .withDescription( + "If the delivery guarantee is configured as " + + DeliveryGuarantee.EXACTLY_ONCE + + " this value is used a prefix for the identifier of all opened Kafka transactions."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** Strategies to derive the data type of a value format by considering a key format. */ + public enum ValueFieldsStrategy { + ALL, + EXCEPT_KEY + } + + /** Startup mode for the Kafka consumer, see {@link #SCAN_STARTUP_MODE}. */ + public enum ScanStartupMode implements DescribedEnum { + + EARLIEST_OFFSET("earliest-offset", text("Start from the earliest offset possible.")), + LATEST_OFFSET("latest-offset", text("Start from the latest offset.")), + GROUP_OFFSETS( + "group-offsets", + text( + "Start from committed offsets in ZooKeeper / Kafka brokers of a specific consumer group.")), + TIMESTAMP("timestamp", text("Start from user-supplied timestamp for each partition.")), + SPECIFIC_OFFSETS( + "specific-offsets", + text("Start from user-supplied specific offsets for each partition.")); + + private final String value; + private final InlineElement description; + + ScanStartupMode(String value, InlineElement description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return description; + } + } + + /** Bounded mode for the Kafka consumer, see {@link #SCAN_BOUNDED_MODE}. */ + public enum ScanBoundedMode implements DescribedEnum { + + UNBOUNDED("unbounded", text("Do not stop consuming")), + LATEST_OFFSET( + "latest-offset", + text( + "Bounded by latest offsets. This is evaluated at the start of consumption" + + " from a given partition.")), + GROUP_OFFSETS( + "group-offsets", + text( + "Bounded by committed offsets in ZooKeeper / Kafka brokers of a specific" + + " consumer group. This is evaluated at the start of consumption" + + " from a given partition.")), + TIMESTAMP("timestamp", text("Bounded by a user-supplied timestamp.")), + SPECIFIC_OFFSETS( + "specific-offsets", + text( + "Bounded by user-supplied specific offsets for each partition. If an offset" + + " for a partition is not provided it will not consume from that" + + " partition.")); + private final String value; + private final InlineElement description; + + ScanBoundedMode(String value, InlineElement description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return description; + } + } + + private KafkaConnectorOptions() { + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java new file mode 100644 index 00000000000..91dfd73f6be --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java @@ -0,0 +1,685 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ValueFieldsStrategy; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.regex.Pattern; +import java.util.stream.IntStream; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; +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; + +/** Utilities for {@link KafkaConnectorOptions}. */ +@Internal +class KafkaConnectorOptionsUtil { + + private static final ConfigOption SCHEMA_REGISTRY_SUBJECT = + ConfigOptions.key("schema-registry.subject").stringType().noDefaultValue(); + + // -------------------------------------------------------------------------------------------- + // Option enumerations + // -------------------------------------------------------------------------------------------- + + // Sink partitioner. + public static final String SINK_PARTITIONER_VALUE_DEFAULT = "default"; + public static final String SINK_PARTITIONER_VALUE_FIXED = "fixed"; + public static final String SINK_PARTITIONER_VALUE_ROUND_ROBIN = "round-robin"; + + // Prefix for Kafka specific properties. + public static final String PROPERTIES_PREFIX = "properties."; + + // Other keywords. + private static final String PARTITION = "partition"; + private static final String OFFSET = "offset"; + protected static final String AVRO_CONFLUENT = "avro-confluent"; + protected static final String DEBEZIUM_AVRO_CONFLUENT = "debezium-avro-confluent"; + private static final List SCHEMA_REGISTRY_FORMATS = + Arrays.asList(AVRO_CONFLUENT, DEBEZIUM_AVRO_CONFLUENT); + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateTopic(tableOptions); + validateScanStartupMode(tableOptions); + validateScanBoundedMode(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateTopic(tableOptions); + validateSinkPartitioner(tableOptions); + } + + public static void validateTopic(ReadableConfig tableOptions) { + Optional> topic = tableOptions.getOptional(TOPIC); + Optional pattern = tableOptions.getOptional(TOPIC_PATTERN); + + if (topic.isPresent() && pattern.isPresent()) { + throw new ValidationException( + "Option 'topic' and 'topic-pattern' shouldn't be set together."); + } + + if (!topic.isPresent() && !pattern.isPresent()) { + throw new ValidationException("Either 'topic' or 'topic-pattern' must be set."); + } + } + + private static void validateScanStartupMode(ReadableConfig tableOptions) { + tableOptions + .getOptional(SCAN_STARTUP_MODE) + .ifPresent( + mode -> { + switch (mode) { + case TIMESTAMP: + if (!tableOptions + .getOptional(SCAN_STARTUP_TIMESTAMP_MILLIS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' startup mode" + + " but missing.", + SCAN_STARTUP_TIMESTAMP_MILLIS.key(), + ScanStartupMode.TIMESTAMP)); + } + + break; + case SPECIFIC_OFFSETS: + if (!tableOptions + .getOptional(SCAN_STARTUP_SPECIFIC_OFFSETS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' startup mode" + + " but missing.", + SCAN_STARTUP_SPECIFIC_OFFSETS.key(), + ScanStartupMode.SPECIFIC_OFFSETS)); + } + if (!isSingleTopic(tableOptions)) { + throw new ValidationException( + "Currently Kafka source only supports specific offset for single topic."); + } + String specificOffsets = + tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS); + parseSpecificOffsets( + specificOffsets, SCAN_STARTUP_SPECIFIC_OFFSETS.key()); + + break; + } + }); + } + + static void validateScanBoundedMode(ReadableConfig tableOptions) { + tableOptions + .getOptional(SCAN_BOUNDED_MODE) + .ifPresent( + mode -> { + switch (mode) { + case TIMESTAMP: + if (!tableOptions + .getOptional(SCAN_BOUNDED_TIMESTAMP_MILLIS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' bounded mode" + + " but missing.", + SCAN_BOUNDED_TIMESTAMP_MILLIS.key(), + ScanBoundedMode.TIMESTAMP)); + } + + break; + case SPECIFIC_OFFSETS: + if (!tableOptions + .getOptional(SCAN_BOUNDED_SPECIFIC_OFFSETS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' bounded mode" + + " but missing.", + SCAN_BOUNDED_SPECIFIC_OFFSETS.key(), + ScanBoundedMode.SPECIFIC_OFFSETS)); + } + if (!isSingleTopic(tableOptions)) { + throw new ValidationException( + "Currently Kafka source only supports specific offset for single topic."); + } + String specificOffsets = + tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS); + parseSpecificOffsets( + specificOffsets, SCAN_BOUNDED_SPECIFIC_OFFSETS.key()); + break; + } + }); + } + + private static void validateSinkPartitioner(ReadableConfig tableOptions) { + tableOptions + .getOptional(SINK_PARTITIONER) + .ifPresent( + partitioner -> { + if (partitioner.equals(SINK_PARTITIONER_VALUE_ROUND_ROBIN) + && tableOptions.getOptional(KEY_FIELDS).isPresent()) { + throw new ValidationException( + "Currently 'round-robin' partitioner only works when option 'key.fields' is not specified."); + } else if (partitioner.isEmpty()) { + throw new ValidationException( + String.format( + "Option '%s' should be a non-empty string.", + SINK_PARTITIONER.key())); + } + }); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + public static List getTopics(ReadableConfig tableOptions) { + return tableOptions.getOptional(TOPIC).orElse(null); + } + + public static Pattern getTopicPattern(ReadableConfig tableOptions) { + return tableOptions.getOptional(TOPIC_PATTERN).map(Pattern::compile).orElse(null); + } + + private static boolean isSingleTopic(ReadableConfig tableOptions) { + // Option 'topic-pattern' is regarded as multi-topics. + return tableOptions.getOptional(TOPIC).map(t -> t.size() == 1).orElse(false); + } + + public static StartupOptions getStartupOptions(ReadableConfig tableOptions) { + final Map specificOffsets = new HashMap<>(); + final StartupMode startupMode = + tableOptions + .getOptional(SCAN_STARTUP_MODE) + .map(KafkaConnectorOptionsUtil::fromOption) + .orElse(StartupMode.GROUP_OFFSETS); + if (startupMode == StartupMode.SPECIFIC_OFFSETS) { + // It will be refactored after support specific offset for multiple topics in + // FLINK-18602. We have already checked tableOptions.get(TOPIC) contains one topic in + // validateScanStartupMode(). + buildSpecificOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets); + } + + final StartupOptions options = new StartupOptions(); + options.startupMode = startupMode; + options.specificOffsets = specificOffsets; + if (startupMode == StartupMode.TIMESTAMP) { + options.startupTimestampMillis = tableOptions.get(SCAN_STARTUP_TIMESTAMP_MILLIS); + } + return options; + } + + public static BoundedOptions getBoundedOptions(ReadableConfig tableOptions) { + final Map specificOffsets = new HashMap<>(); + final BoundedMode boundedMode = + KafkaConnectorOptionsUtil.fromOption(tableOptions.get(SCAN_BOUNDED_MODE)); + if (boundedMode == BoundedMode.SPECIFIC_OFFSETS) { + buildBoundedOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets); + } + + final BoundedOptions options = new BoundedOptions(); + options.boundedMode = boundedMode; + options.specificOffsets = specificOffsets; + if (boundedMode == BoundedMode.TIMESTAMP) { + options.boundedTimestampMillis = tableOptions.get(SCAN_BOUNDED_TIMESTAMP_MILLIS); + } + return options; + } + + private static void buildSpecificOffsets( + ReadableConfig tableOptions, + String topic, + Map specificOffsets) { + String specificOffsetsStrOpt = tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS); + final Map offsetMap = + parseSpecificOffsets(specificOffsetsStrOpt, SCAN_STARTUP_SPECIFIC_OFFSETS.key()); + offsetMap.forEach( + (partition, offset) -> { + final KafkaTopicPartition topicPartition = + new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + } + + public static void buildBoundedOffsets( + ReadableConfig tableOptions, + String topic, + Map specificOffsets) { + String specificOffsetsEndOpt = tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS); + final Map offsetMap = + parseSpecificOffsets(specificOffsetsEndOpt, SCAN_BOUNDED_SPECIFIC_OFFSETS.key()); + + offsetMap.forEach( + (partition, offset) -> { + final KafkaTopicPartition topicPartition = + new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + } + + /** + * Returns the {@link StartupMode} of Kafka Consumer by passed-in table-specific {@link + * ScanStartupMode}. + */ + private static StartupMode fromOption(ScanStartupMode scanStartupMode) { + switch (scanStartupMode) { + case EARLIEST_OFFSET: + return StartupMode.EARLIEST; + case LATEST_OFFSET: + return StartupMode.LATEST; + case GROUP_OFFSETS: + return StartupMode.GROUP_OFFSETS; + case SPECIFIC_OFFSETS: + return StartupMode.SPECIFIC_OFFSETS; + case TIMESTAMP: + return StartupMode.TIMESTAMP; + + default: + throw new TableException( + "Unsupported startup mode. Validator should have checked that."); + } + } + + /** + * Returns the {@link BoundedMode} of Kafka Consumer by passed-in table-specific {@link + * ScanBoundedMode}. + */ + private static BoundedMode fromOption(ScanBoundedMode scanBoundedMode) { + switch (scanBoundedMode) { + case UNBOUNDED: + return BoundedMode.UNBOUNDED; + case LATEST_OFFSET: + return BoundedMode.LATEST; + case GROUP_OFFSETS: + return BoundedMode.GROUP_OFFSETS; + case TIMESTAMP: + return BoundedMode.TIMESTAMP; + case SPECIFIC_OFFSETS: + return BoundedMode.SPECIFIC_OFFSETS; + + default: + throw new TableException( + "Unsupported bounded mode. Validator should have checked that."); + } + } + + public static Properties getKafkaProperties(Map tableOptions) { + final Properties kafkaProperties = new Properties(); + + if (hasKafkaClientProperties(tableOptions)) { + tableOptions.keySet().stream() + .filter(key -> key.startsWith(PROPERTIES_PREFIX)) + .forEach( + key -> { + final String value = tableOptions.get(key); + final String subKey = key.substring((PROPERTIES_PREFIX).length()); + kafkaProperties.put(subKey, value); + }); + } + return kafkaProperties; + } + + /** + * The partitioner can be either "fixed", "round-robin" or a customized partitioner full class + * name. + */ + public static Optional> getFlinkKafkaPartitioner( + ReadableConfig tableOptions, ClassLoader classLoader) { + return tableOptions + .getOptional(SINK_PARTITIONER) + .flatMap( + (String partitioner) -> { + switch (partitioner) { + case SINK_PARTITIONER_VALUE_FIXED: + return Optional.of(new FlinkFixedPartitioner<>()); + case SINK_PARTITIONER_VALUE_DEFAULT: + case SINK_PARTITIONER_VALUE_ROUND_ROBIN: + return Optional.empty(); + // Default fallback to full class name of the partitioner. + default: + return Optional.of( + initializePartitioner(partitioner, classLoader)); + } + }); + } + + /** + * Parses specificOffsets String to Map. + * + *

specificOffsets String format was given as following: + * + *

+     *     scan.startup.specific-offsets = partition:0,offset:42;partition:1,offset:300
+     * 
+ * + * @return specificOffsets with Map format, key is partition, and value is offset + */ + public static Map parseSpecificOffsets( + String specificOffsetsStr, String optionKey) { + final Map offsetMap = new HashMap<>(); + final String[] pairs = specificOffsetsStr.split(";"); + final String validationExceptionMessage = + String.format( + "Invalid properties '%s' should follow the format " + + "'partition:0,offset:42;partition:1,offset:300', but is '%s'.", + optionKey, specificOffsetsStr); + + if (pairs.length == 0) { + throw new ValidationException(validationExceptionMessage); + } + + for (String pair : pairs) { + if (null == pair || pair.length() == 0 || !pair.contains(",")) { + throw new ValidationException(validationExceptionMessage); + } + + final String[] kv = pair.split(","); + if (kv.length != 2 + || !kv[0].startsWith(PARTITION + ':') + || !kv[1].startsWith(OFFSET + ':')) { + throw new ValidationException(validationExceptionMessage); + } + + String partitionValue = kv[0].substring(kv[0].indexOf(":") + 1); + String offsetValue = kv[1].substring(kv[1].indexOf(":") + 1); + try { + final Integer partition = Integer.valueOf(partitionValue); + final Long offset = Long.valueOf(offsetValue); + offsetMap.put(partition, offset); + } catch (NumberFormatException e) { + throw new ValidationException(validationExceptionMessage, e); + } + } + return offsetMap; + } + + /** + * Decides if the table options contains Kafka client properties that start with prefix + * 'properties'. + */ + private static boolean hasKafkaClientProperties(Map tableOptions) { + return tableOptions.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX)); + } + + /** Returns a class value with the given class name. */ + private static KafkaPartitioner initializePartitioner( + String name, ClassLoader classLoader) { + try { + Class clazz = Class.forName(name, true, classLoader); + if (!KafkaPartitioner.class.isAssignableFrom(clazz)) { + throw new ValidationException( + String.format( + "Sink partitioner class '%s' should implement the required class %s", + name, KafkaPartitioner.class.getName())); + } + @SuppressWarnings("unchecked") + final KafkaPartitioner kafkaPartitioner = + InstantiationUtil.instantiate(name, KafkaPartitioner.class, classLoader); + + return kafkaPartitioner; + } catch (ClassNotFoundException | FlinkException e) { + throw new ValidationException( + String.format("Could not find and instantiate partitioner class '%s'", name), + e); + } + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the key format and the order that those fields have in the key format. + * + *

See {@link KafkaConnectorOptions#KEY_FORMAT}, {@link KafkaConnectorOptions#KEY_FIELDS}, + * and {@link KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information. + */ + public static int[] createKeyFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); + + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + + if (!optionalKeyFormat.isPresent()) { + return new int[0]; + } + + final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse(""); + + final List keyFields = optionalKeyFields.get(); + final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType); + return keyFields.stream() + .mapToInt( + keyField -> { + final int pos = physicalFields.indexOf(keyField); + // check that field name exists + if (pos < 0) { + throw new ValidationException( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option:\n" + + "%s", + keyField, KEY_FIELDS.key(), physicalFields)); + } + // check that field name is prefixed correctly + if (!keyField.startsWith(keyPrefix)) { + throw new ValidationException( + String.format( + "All fields in '%s' must be prefixed with '%s' when option '%s' " + + "is set but field '%s' is not prefixed.", + KEY_FIELDS.key(), + keyPrefix, + KEY_FIELDS_PREFIX.key(), + keyField)); + } + return pos; + }) + .toArray(); + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the value format. + * + *

See {@link KafkaConnectorOptions#VALUE_FORMAT}, {@link + * KafkaConnectorOptions#VALUE_FIELDS_INCLUDE}, and {@link + * KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information. + */ + public static int[] createValueFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); + final IntStream physicalFields = IntStream.range(0, physicalFieldCount); + + final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse(""); + + final ValueFieldsStrategy strategy = options.get(VALUE_FIELDS_INCLUDE); + if (strategy == ValueFieldsStrategy.ALL) { + if (keyPrefix.length() > 0) { + throw new ValidationException( + String.format( + "A key prefix is not allowed when option '%s' is set to '%s'. " + + "Set it to '%s' instead to avoid field overlaps.", + VALUE_FIELDS_INCLUDE.key(), + ValueFieldsStrategy.ALL, + ValueFieldsStrategy.EXCEPT_KEY)); + } + return physicalFields.toArray(); + } else if (strategy == ValueFieldsStrategy.EXCEPT_KEY) { + final int[] keyProjection = createKeyFormatProjection(options, physicalDataType); + return physicalFields + .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos)) + .toArray(); + } + throw new TableException("Unknown value fields strategy:" + strategy); + } + + /** + * Returns a new table context with a default schema registry subject value in the options if + * the format is a schema registry format (e.g. 'avro-confluent') and the subject is not + * defined. + */ + public static DynamicTableFactory.Context autoCompleteSchemaRegistrySubject( + DynamicTableFactory.Context context) { + Map tableOptions = context.getCatalogTable().getOptions(); + Map newOptions = autoCompleteSchemaRegistrySubject(tableOptions); + if (newOptions.size() > tableOptions.size()) { + // build a new context + return new FactoryUtil.DefaultDynamicTableContext( + context.getObjectIdentifier(), + context.getCatalogTable().copy(newOptions), + context.getEnrichmentOptions(), + context.getConfiguration(), + context.getClassLoader(), + context.isTemporary()); + } else { + return context; + } + } + + private static Map autoCompleteSchemaRegistrySubject( + Map options) { + Configuration configuration = Configuration.fromMap(options); + // the subject autoComplete should only be used in sink with a single topic, check the topic + // option first + validateTopic(configuration); + if (configuration.contains(TOPIC) && isSingleTopic(configuration)) { + final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); + final Optional keyFormat = configuration.getOptional(KEY_FORMAT); + final Optional format = configuration.getOptional(FORMAT); + final String topic = configuration.get(TOPIC).get(0); + + if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { + autoCompleteSubject(configuration, format.get(), topic + "-value"); + } else if (valueFormat.isPresent() + && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { + autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); + } + + if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { + autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + } + } + return configuration.toMap(); + } + + private static void autoCompleteSubject( + Configuration configuration, String format, String subject) { + ConfigOption subjectOption = + ConfigOptions.key(format + "." + SCHEMA_REGISTRY_SUBJECT.key()) + .stringType() + .noDefaultValue(); + if (!configuration.getOptional(subjectOption).isPresent()) { + configuration.setString(subjectOption, subject); + } + } + + static void validateDeliveryGuarantee(ReadableConfig tableOptions) { + if (tableOptions.get(DELIVERY_GUARANTEE) == DeliveryGuarantee.EXACTLY_ONCE + && !tableOptions.getOptional(TRANSACTIONAL_ID_PREFIX).isPresent()) { + throw new ValidationException( + TRANSACTIONAL_ID_PREFIX.key() + + " must be specified when using DeliveryGuarantee.EXACTLY_ONCE."); + } + } + + // -------------------------------------------------------------------------------------------- + // Inner classes + // -------------------------------------------------------------------------------------------- + + /** Kafka startup options. * */ + public static class StartupOptions { + + public StartupMode startupMode; + public Map specificOffsets; + public long startupTimestampMillis; + } + + /** Kafka bounded options. * */ + public static class BoundedOptions { + + public BoundedMode boundedMode; + public Map specificOffsets; + public long boundedTimestampMillis; + } + + private KafkaConnectorOptionsUtil() { + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java new file mode 100644 index 00000000000..77f74959c58 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java @@ -0,0 +1,511 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; +import org.apache.flink.connector.kafka.sink.KafkaSink; +import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.header.Header; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +import static org.apache.inlong.sort.kafka.util.Preconditions.checkNotNull; + +/** A version-agnostic Kafka {@link DynamicTableSink}. */ +@Internal +public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetadata { + + private static final String UPSERT_KAFKA_TRANSFORMATION = "upsert-kafka"; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** Metadata that is appended at the end of a physical sink row. */ + protected List metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + /** Data type of consumed data type. */ + protected DataType consumedDataType; + + /** Data type to configure the formats. */ + protected final DataType physicalDataType; + + /** Optional format for encoding keys to Kafka. */ + protected final @Nullable EncodingFormat> keyEncodingFormat; + + /** Format for encoding values to Kafka. */ + protected final EncodingFormat> valueEncodingFormat; + + /** Indices that determine the key fields and the source position in the consumed row. */ + protected final int[] keyProjection; + + /** Indices that determine the value fields and the source position in the consumed row. */ + protected final int[] valueProjection; + + /** Prefix that needs to be removed from fields when constructing the physical data type. */ + protected final @Nullable String keyPrefix; + + // -------------------------------------------------------------------------------------------- + // Kafka-specific attributes + // -------------------------------------------------------------------------------------------- + + /** The defined delivery guarantee. */ + private final DeliveryGuarantee deliveryGuarantee; + + /** + * If the {@link #deliveryGuarantee} is {@link DeliveryGuarantee#EXACTLY_ONCE} the value is the + * prefix for all ids of opened Kafka transactions. + */ + @Nullable + private final String transactionalIdPrefix; + + /** The Kafka topics to allow for producing. */ + protected final List topics; + + /** The Kafka topic pattern of topics allowed to produce to. */ + protected final Pattern topicPattern; + + /** Properties for the Kafka producer. */ + protected final Properties properties; + + /** Partitioner to select Kafka partition for each item. */ + protected final @Nullable KafkaPartitioner partitioner; + + /** + * Flag to determine sink mode. In upsert mode sink transforms the delete/update-before message + * to tombstone message. + */ + protected final boolean upsertMode; + + /** Sink buffer flush config which only supported in upsert mode now. */ + protected final SinkBufferFlushMode flushMode; + + /** Parallelism of the physical Kafka producer. * */ + protected final @Nullable Integer parallelism; + + public KafkaDynamicSink( + DataType consumedDataType, + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + EncodingFormat> valueEncodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + @Nullable KafkaPartitioner partitioner, + DeliveryGuarantee deliveryGuarantee, + boolean upsertMode, + SinkBufferFlushMode flushMode, + @Nullable Integer parallelism, + @Nullable String transactionalIdPrefix) { + // Format attributes + this.consumedDataType = + checkNotNull(consumedDataType, "Consumed data type must not be null."); + this.physicalDataType = + checkNotNull(physicalDataType, "Physical data type must not be null."); + this.keyEncodingFormat = keyEncodingFormat; + this.valueEncodingFormat = + checkNotNull(valueEncodingFormat, "Value encoding format must not be null."); + this.keyProjection = checkNotNull(keyProjection, "Key projection must not be null."); + this.valueProjection = checkNotNull(valueProjection, "Value projection must not be null."); + this.keyPrefix = keyPrefix; + this.transactionalIdPrefix = transactionalIdPrefix; + // Mutable attributes + this.metadataKeys = Collections.emptyList(); + // Kafka-specific attributes + this.topics = topics; + this.topicPattern = topicPattern; + this.properties = checkNotNull(properties, "Properties must not be null."); + this.partitioner = partitioner; + this.deliveryGuarantee = + checkNotNull(deliveryGuarantee, "DeliveryGuarantee must not be null."); + this.upsertMode = upsertMode; + this.flushMode = checkNotNull(flushMode); + if (flushMode.isEnabled() && !upsertMode) { + throw new IllegalArgumentException( + "Sink buffer flush is only supported in upsert-kafka."); + } + this.parallelism = parallelism; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return valueEncodingFormat.getChangelogMode(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, keyPrefix); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + + final KafkaSinkBuilder sinkBuilder = KafkaSink.builder(); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + if (transactionalIdPrefix != null) { + sinkBuilder.setTransactionalIdPrefix(transactionalIdPrefix); + } + final KafkaSink kafkaSink = + sinkBuilder + .setDeliveryGuarantee(deliveryGuarantee) + .setBootstrapServers( + properties.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG).toString()) + .setKafkaProducerConfig(properties) + .setRecordSerializer( + new DynamicKafkaRecordSerializationSchema( + topics, + topicPattern, + partitioner, + keySerialization, + valueSerialization, + getFieldGetters(physicalChildren, keyProjection), + getFieldGetters(physicalChildren, valueProjection), + hasMetadata(), + getMetadataPositions(physicalChildren), + upsertMode)) + .build(); + if (flushMode.isEnabled() && upsertMode) { + return new DataStreamSinkProvider() { + + @Override + public DataStreamSink consumeDataStream( + ProviderContext providerContext, DataStream dataStream) { + final boolean objectReuse = + dataStream.getExecutionEnvironment().getConfig().isObjectReuseEnabled(); + final ReducingUpsertSink sink = + new ReducingUpsertSink<>( + kafkaSink, + physicalDataType, + keyProjection, + flushMode, + objectReuse + ? createRowDataTypeSerializer( + context, + dataStream.getExecutionConfig())::copy + : rowData -> rowData); + final DataStreamSink end = dataStream.sinkTo(sink); + providerContext.generateUid(UPSERT_KAFKA_TRANSFORMATION).ifPresent(end::uid); + if (parallelism != null) { + end.setParallelism(parallelism); + } + return end; + } + }; + } + return SinkV2Provider.of(kafkaSink, parallelism); + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + for (WritableMetadata m : WritableMetadata.values()) { + if (topics != null && topics.size() == 1 && WritableMetadata.TOPIC.key.equals(m.key)) { + // When `topic` is a singleton list, TOPIC metadata is not writable + continue; + } + metadataMap.put(m.key, m.dataType); + } + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.metadataKeys = metadataKeys; + this.consumedDataType = consumedDataType; + } + + @Override + public DynamicTableSink copy() { + final KafkaDynamicSink copy = + new KafkaDynamicSink( + consumedDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + partitioner, + deliveryGuarantee, + upsertMode, + flushMode, + parallelism, + transactionalIdPrefix); + copy.metadataKeys = metadataKeys; + return copy; + } + + @Override + public String asSummaryString() { + return "Kafka table sink"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final KafkaDynamicSink that = (KafkaDynamicSink) o; + return Objects.equals(metadataKeys, that.metadataKeys) + && Objects.equals(consumedDataType, that.consumedDataType) + && Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(keyPrefix, that.keyPrefix) + && Objects.equals(topics, that.topics) + && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern)) + && Objects.equals(properties, that.properties) + && Objects.equals(partitioner, that.partitioner) + && Objects.equals(deliveryGuarantee, that.deliveryGuarantee) + && Objects.equals(upsertMode, that.upsertMode) + && Objects.equals(flushMode, that.flushMode) + && Objects.equals(transactionalIdPrefix, that.transactionalIdPrefix) + && Objects.equals(parallelism, that.parallelism); + } + + @Override + public int hashCode() { + return Objects.hash( + metadataKeys, + consumedDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + partitioner, + deliveryGuarantee, + upsertMode, + flushMode, + transactionalIdPrefix, + parallelism); + } + + // -------------------------------------------------------------------------------------------- + + private TypeSerializer createRowDataTypeSerializer( + Context context, ExecutionConfig executionConfig) { + final TypeInformation typeInformation = + context.createTypeInformation(consumedDataType); + return typeInformation.createSerializer(executionConfig); + } + + private int[] getMetadataPositions(List physicalChildren) { + return Stream.of(WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = metadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildren.size() + pos; + }) + .toArray(); + } + + private boolean hasMetadata() { + return metadataKeys.size() > 0; + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] keyProjection) { + return Arrays.stream(keyProjection) + .mapToObj( + targetField -> RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + private @Nullable SerializationSchema createSerialization( + Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + enum WritableMetadata { + + TOPIC( + "topic", + DataTypes.STRING().notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return row.getString(pos).toString(); + } + }), + HEADERS( + "headers", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable()) + .nullable(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + final List

headers = new ArrayList<>(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final byte[] value = valueArray.getBinary(i); + headers.add(new KafkaHeader(key, value)); + } + } + return headers; + } + }), + + TIMESTAMP( + "timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + } + }); + + final String key; + + final DataType dataType; + + final MetadataConverter converter; + + WritableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } + + interface MetadataConverter extends Serializable { + + Object read(RowData consumedRow, int pos); + } + + // -------------------------------------------------------------------------------------------- + + private static class KafkaHeader implements Header { + + private final String key; + + private final byte[] value; + + KafkaHeader(String key, byte[] value) { + this.key = key; + this.value = value; + } + + @Override + public String key() { + return key; + } + + @Override + public byte[] value() { + return value; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java new file mode 100644 index 00000000000..91f5ce5e20b --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java @@ -0,0 +1,684 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.source.KafkaSource; +import org.apache.flink.connector.kafka.source.KafkaSourceBuilder; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.table.DynamicKafkaDeserializationSchema.MetadataConverter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; +import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** A version-agnostic Kafka {@link ScanTableSource}. */ +@Internal +public class KafkaDynamicSource + implements + ScanTableSource, + SupportsReadingMetadata, + SupportsWatermarkPushDown { + + private static final String KAFKA_TRANSFORMATION = "kafka"; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** Data type that describes the final output of the source. */ + protected DataType producedDataType; + + /** Metadata that is appended at the end of a physical source row. */ + protected List metadataKeys; + + /** Watermark strategy that is used to generate per-partition watermark. */ + protected @Nullable WatermarkStrategy watermarkStrategy; + + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String VALUE_METADATA_PREFIX = "value."; + + /** Data type to configure the formats. */ + protected final DataType physicalDataType; + + /** Optional format for decoding keys from Kafka. */ + protected final @Nullable DecodingFormat> keyDecodingFormat; + + /** Format for decoding values from Kafka. */ + protected final DecodingFormat> valueDecodingFormat; + + /** Indices that determine the key fields and the target position in the produced row. */ + protected final int[] keyProjection; + + /** Indices that determine the value fields and the target position in the produced row. */ + protected final int[] valueProjection; + + /** Prefix that needs to be removed from fields when constructing the physical data type. */ + protected final @Nullable String keyPrefix; + + // -------------------------------------------------------------------------------------------- + // Kafka-specific attributes + // -------------------------------------------------------------------------------------------- + + /** The Kafka topics to consume. */ + protected final List topics; + + /** The Kafka topic pattern to consume. */ + protected final Pattern topicPattern; + + /** Properties for the Kafka consumer. */ + protected final Properties properties; + + /** + * The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}). + */ + protected final StartupMode startupMode; + + /** + * Specific startup offsets; only relevant when startup mode is {@link + * StartupMode#SPECIFIC_OFFSETS}. + */ + protected final Map specificStartupOffsets; + + /** + * The start timestamp to locate partition offsets; only relevant when startup mode is {@link + * StartupMode#TIMESTAMP}. + */ + protected final long startupTimestampMillis; + + /** The bounded mode for the contained consumer (default is an unbounded data stream). */ + protected final BoundedMode boundedMode; + + /** + * Specific end offsets; only relevant when bounded mode is {@link + * BoundedMode#SPECIFIC_OFFSETS}. + */ + protected final Map specificBoundedOffsets; + + /** + * The bounded timestamp to locate partition offsets; only relevant when bounded mode is {@link + * BoundedMode#TIMESTAMP}. + */ + protected final long boundedTimestampMillis; + + /** Flag to determine source mode. In upsert mode, it will keep the tombstone message. * */ + protected final boolean upsertMode; + + protected final String tableIdentifier; + + public KafkaDynamicSource( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + DecodingFormat> valueDecodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + StartupMode startupMode, + Map specificStartupOffsets, + long startupTimestampMillis, + BoundedMode boundedMode, + Map specificBoundedOffsets, + long boundedTimestampMillis, + boolean upsertMode, + String tableIdentifier) { + // Format attributes + this.physicalDataType = + Preconditions.checkNotNull( + physicalDataType, "Physical data type must not be null."); + this.keyDecodingFormat = keyDecodingFormat; + this.valueDecodingFormat = + Preconditions.checkNotNull( + valueDecodingFormat, "Value decoding format must not be null."); + this.keyProjection = + Preconditions.checkNotNull(keyProjection, "Key projection must not be null."); + this.valueProjection = + Preconditions.checkNotNull(valueProjection, "Value projection must not be null."); + this.keyPrefix = keyPrefix; + // Mutable attributes + this.producedDataType = physicalDataType; + this.metadataKeys = Collections.emptyList(); + this.watermarkStrategy = null; + // Kafka-specific attributes + Preconditions.checkArgument( + (topics != null && topicPattern == null) + || (topics == null && topicPattern != null), + "Either Topic or Topic Pattern must be set for source."); + this.topics = topics; + this.topicPattern = topicPattern; + this.properties = Preconditions.checkNotNull(properties, "Properties must not be null."); + this.startupMode = + Preconditions.checkNotNull(startupMode, "Startup mode must not be null."); + this.specificStartupOffsets = + Preconditions.checkNotNull( + specificStartupOffsets, "Specific offsets must not be null."); + this.startupTimestampMillis = startupTimestampMillis; + this.boundedMode = + Preconditions.checkNotNull(boundedMode, "Bounded mode must not be null."); + this.specificBoundedOffsets = + Preconditions.checkNotNull( + specificBoundedOffsets, "Specific bounded offsets must not be null."); + this.boundedTimestampMillis = boundedTimestampMillis; + this.upsertMode = upsertMode; + this.tableIdentifier = tableIdentifier; + } + + @Override + public ChangelogMode getChangelogMode() { + return valueDecodingFormat.getChangelogMode(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + final DeserializationSchema keyDeserialization = + createDeserialization(context, keyDecodingFormat, keyProjection, keyPrefix); + + final DeserializationSchema valueDeserialization = + createDeserialization(context, valueDecodingFormat, valueProjection, null); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + final KafkaSource kafkaSource = + createKafkaSource(keyDeserialization, valueDeserialization, producedTypeInfo); + + return new DataStreamScanProvider() { + + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + if (watermarkStrategy == null) { + watermarkStrategy = WatermarkStrategy.noWatermarks(); + } + DataStreamSource sourceStream = + execEnv.fromSource( + kafkaSource, watermarkStrategy, "KafkaSource-" + tableIdentifier); + providerContext.generateUid(KAFKA_TRANSFORMATION).ifPresent(sourceStream::uid); + return sourceStream; + } + + @Override + public boolean isBounded() { + return kafkaSource.getBoundedness() == Boundedness.BOUNDED; + } + }; + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + + // according to convention, the order of the final row must be + // PHYSICAL + FORMAT METADATA + CONNECTOR METADATA + // where the format metadata has highest precedence + + // add value format metadata with prefix + valueDecodingFormat + .listReadableMetadata() + .forEach((key, value) -> metadataMap.put(VALUE_METADATA_PREFIX + key, value)); + + // add connector metadata + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.putIfAbsent(m.key, m.dataType)); + + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + metadataKeys.stream() + .filter(k -> k.startsWith(VALUE_METADATA_PREFIX)) + .collect(Collectors.toList()); + final List connectorMetadataKeys = new ArrayList<>(metadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = valueDecodingFormat.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(VALUE_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + valueDecodingFormat.applyReadableMetadata(requestedFormatMetadataKeys); + } + + this.metadataKeys = connectorMetadataKeys; + this.producedDataType = producedDataType; + } + + @Override + public boolean supportsMetadataProjection() { + return false; + } + + @Override + public void applyWatermark(WatermarkStrategy watermarkStrategy) { + this.watermarkStrategy = watermarkStrategy; + } + + @Override + public DynamicTableSource copy() { + final KafkaDynamicSource copy = + new KafkaDynamicSource( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificBoundedOffsets, + boundedTimestampMillis, + upsertMode, + tableIdentifier); + copy.producedDataType = producedDataType; + copy.metadataKeys = metadataKeys; + copy.watermarkStrategy = watermarkStrategy; + return copy; + } + + @Override + public String asSummaryString() { + return "Kafka table source"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final KafkaDynamicSource that = (KafkaDynamicSource) o; + return Objects.equals(producedDataType, that.producedDataType) + && Objects.equals(metadataKeys, that.metadataKeys) + && Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyDecodingFormat, that.keyDecodingFormat) + && Objects.equals(valueDecodingFormat, that.valueDecodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(keyPrefix, that.keyPrefix) + && Objects.equals(topics, that.topics) + && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern)) + && Objects.equals(properties, that.properties) + && startupMode == that.startupMode + && Objects.equals(specificStartupOffsets, that.specificStartupOffsets) + && startupTimestampMillis == that.startupTimestampMillis + && boundedMode == that.boundedMode + && Objects.equals(specificBoundedOffsets, that.specificBoundedOffsets) + && boundedTimestampMillis == that.boundedTimestampMillis + && Objects.equals(upsertMode, that.upsertMode) + && Objects.equals(tableIdentifier, that.tableIdentifier) + && Objects.equals(watermarkStrategy, that.watermarkStrategy); + } + + @Override + public int hashCode() { + return Objects.hash( + producedDataType, + metadataKeys, + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + Arrays.hashCode(keyProjection), + Arrays.hashCode(valueProjection), + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificBoundedOffsets, + boundedTimestampMillis, + upsertMode, + tableIdentifier, + watermarkStrategy); + } + + // -------------------------------------------------------------------------------------------- + + protected KafkaSource createKafkaSource( + DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo) { + + final KafkaDeserializationSchema kafkaDeserializer = + createKafkaDeserializationSchema( + keyDeserialization, valueDeserialization, producedTypeInfo); + + final KafkaSourceBuilder kafkaSourceBuilder = KafkaSource.builder(); + + if (topics != null) { + kafkaSourceBuilder.setTopics(topics); + } else { + kafkaSourceBuilder.setTopicPattern(topicPattern); + } + + switch (startupMode) { + case EARLIEST: + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.earliest()); + break; + case LATEST: + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.latest()); + break; + case GROUP_OFFSETS: + String offsetResetConfig = + properties.getProperty( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + OffsetResetStrategy.NONE.name()); + OffsetResetStrategy offsetResetStrategy = getResetStrategy(offsetResetConfig); + kafkaSourceBuilder.setStartingOffsets( + OffsetsInitializer.committedOffsets(offsetResetStrategy)); + break; + case SPECIFIC_OFFSETS: + Map offsets = new HashMap<>(); + specificStartupOffsets.forEach( + (tp, offset) -> offsets.put( + new TopicPartition(tp.getTopic(), tp.getPartition()), + offset)); + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.offsets(offsets)); + break; + case TIMESTAMP: + kafkaSourceBuilder.setStartingOffsets( + OffsetsInitializer.timestamp(startupTimestampMillis)); + break; + } + + switch (boundedMode) { + case UNBOUNDED: + kafkaSourceBuilder.setUnbounded(new NoStoppingOffsetsInitializer()); + break; + case LATEST: + kafkaSourceBuilder.setBounded(OffsetsInitializer.latest()); + break; + case GROUP_OFFSETS: + kafkaSourceBuilder.setBounded(OffsetsInitializer.committedOffsets()); + break; + case SPECIFIC_OFFSETS: + Map offsets = new HashMap<>(); + specificBoundedOffsets.forEach( + (tp, offset) -> offsets.put( + new TopicPartition(tp.getTopic(), tp.getPartition()), + offset)); + kafkaSourceBuilder.setBounded(OffsetsInitializer.offsets(offsets)); + break; + case TIMESTAMP: + kafkaSourceBuilder.setBounded(OffsetsInitializer.timestamp(boundedTimestampMillis)); + break; + } + + kafkaSourceBuilder + .setProperties(properties) + .setDeserializer(KafkaRecordDeserializationSchema.of(kafkaDeserializer)); + + return kafkaSourceBuilder.build(); + } + + private OffsetResetStrategy getResetStrategy(String offsetResetConfig) { + return Arrays.stream(OffsetResetStrategy.values()) + .filter(ors -> ors.name().equals(offsetResetConfig.toUpperCase(Locale.ROOT))) + .findAny() + .orElseThrow( + () -> new IllegalArgumentException( + String.format( + "%s can not be set to %s. Valid values: [%s]", + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + offsetResetConfig, + Arrays.stream(OffsetResetStrategy.values()) + .map(Enum::name) + .map(String::toLowerCase) + .collect(Collectors.joining(","))))); + } + + private KafkaDeserializationSchema createKafkaDeserializationSchema( + DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo) { + final MetadataConverter[] metadataConverters = + metadataKeys.stream() + .map( + k -> Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(m -> m.converter) + .toArray(MetadataConverter[]::new); + + // check if connector metadata is used at all + final boolean hasMetadata = metadataKeys.size() > 0; + + // adjust physical arity with value format's metadata + final int adjustedPhysicalArity = + DataType.getFieldDataTypes(producedDataType).size() - metadataKeys.size(); + + // adjust value format projection to include value format's metadata columns at the end + final int[] adjustedValueProjection = + IntStream.concat( + IntStream.of(valueProjection), + IntStream.range( + keyProjection.length + valueProjection.length, + adjustedPhysicalArity)) + .toArray(); + + return new DynamicKafkaDeserializationSchema( + adjustedPhysicalArity, + keyDeserialization, + keyProjection, + valueDeserialization, + adjustedValueProjection, + hasMetadata, + metadataConverters, + producedTypeInfo, + upsertMode); + } + + private @Nullable DeserializationSchema createDeserialization( + Context context, + @Nullable DecodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeDecoder(context, physicalFormatDataType); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + enum ReadableMetadata { + + TOPIC( + "topic", + DataTypes.STRING().notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return StringData.fromString(record.topic()); + } + }), + + PARTITION( + "partition", + DataTypes.INT().notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.partition(); + } + }), + + HEADERS( + "headers", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable()) + .notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + final Map map = new HashMap<>(); + for (Header header : record.headers()) { + map.put(StringData.fromString(header.key()), header.value()); + } + return new GenericMapData(map); + } + }), + + LEADER_EPOCH( + "leader-epoch", + DataTypes.INT().nullable(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.leaderEpoch().orElse(null); + } + }), + + OFFSET( + "offset", + DataTypes.BIGINT().notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.offset(); + } + }), + + TIMESTAMP( + "timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return TimestampData.fromEpochMillis(record.timestamp()); + } + }), + + TIMESTAMP_TYPE( + "timestamp-type", + DataTypes.STRING().notNull(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return StringData.fromString(record.timestampType().toString()); + } + }); + + final String key; + + final DataType dataType; + + final MetadataConverter converter; + + ReadableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java new file mode 100644 index 00000000000..01bf6f468a8 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java @@ -0,0 +1,449 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaPartitioner; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.Format; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.PROPS_GROUP_ID; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SCAN_TOPIC_PARTITION_DISCOVERY; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getFlinkKafkaPartitioner; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getStartupOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getTopics; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.validateTableSinkOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.validateTableSourceOptions; +import static org.apache.inlong.sort.protocol.constant.KafkaConstant.KAFKA; + +/** + * Factory for creating configured instances of {@link KafkaDynamicSource} and {@link + * KafkaDynamicSink}. + */ +@Internal +public class KafkaDynamicTableFactory + implements + DynamicTableSourceFactory, + DynamicTableSinkFactory { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaDynamicTableFactory.class); + private static final ConfigOption SINK_SEMANTIC = + ConfigOptions.key("sink.semantic") + .stringType() + .noDefaultValue() + .withDescription("Optional semantic when committing."); + + public static final String IDENTIFIER = KAFKA; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(PROPS_BOOTSTRAP_SERVERS); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(FactoryUtil.FORMAT); + options.add(KEY_FORMAT); + options.add(KEY_FIELDS); + options.add(KEY_FIELDS_PREFIX); + options.add(VALUE_FORMAT); + options.add(VALUE_FIELDS_INCLUDE); + options.add(TOPIC); + options.add(TOPIC_PATTERN); + options.add(PROPS_GROUP_ID); + options.add(SCAN_STARTUP_MODE); + options.add(SCAN_STARTUP_SPECIFIC_OFFSETS); + options.add(SCAN_TOPIC_PARTITION_DISCOVERY); + options.add(SCAN_STARTUP_TIMESTAMP_MILLIS); + options.add(SINK_PARTITIONER); + options.add(SINK_PARALLELISM); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); + options.add(SINK_SEMANTIC); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + return options; + } + + @Override + public Set> forwardOptions() { + return Stream.of( + PROPS_BOOTSTRAP_SERVERS, + PROPS_GROUP_ID, + TOPIC, + TOPIC_PATTERN, + SCAN_STARTUP_MODE, + SCAN_STARTUP_SPECIFIC_OFFSETS, + SCAN_TOPIC_PARTITION_DISCOVERY, + SCAN_STARTUP_TIMESTAMP_MILLIS, + SINK_PARTITIONER, + SINK_PARALLELISM, + TRANSACTIONAL_ID_PREFIX) + .collect(Collectors.toSet()); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + final TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + final Optional>> keyDecodingFormat = + getKeyDecodingFormat(helper); + + final DecodingFormat> valueDecodingFormat = + getValueDecodingFormat(helper); + + helper.validateExcept(PROPERTIES_PREFIX); + + final ReadableConfig tableOptions = helper.getOptions(); + + validateTableSourceOptions(tableOptions); + + validatePKConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + valueDecodingFormat); + + final KafkaConnectorOptionsUtil.StartupOptions startupOptions = getStartupOptions(tableOptions); + + final KafkaConnectorOptionsUtil.BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + + final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + + // add topic-partition discovery + final Duration partitionDiscoveryInterval = + tableOptions.get(SCAN_TOPIC_PARTITION_DISCOVERY); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + Long.toString(partitionDiscoveryInterval.toMillis())); + + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + + return createKafkaTableSource( + physicalDataType, + keyDecodingFormat.orElse(null), + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + getTopics(tableOptions), + getTopicPattern(tableOptions), + properties, + startupOptions.startupMode, + startupOptions.specificOffsets, + startupOptions.startupTimestampMillis, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, + context.getObjectIdentifier().asSummaryString()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + final TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper( + this, autoCompleteSchemaRegistrySubject(context)); + + final Optional>> keyEncodingFormat = + getKeyEncodingFormat(helper); + + final EncodingFormat> valueEncodingFormat = + getValueEncodingFormat(helper); + + helper.validateExcept(PROPERTIES_PREFIX); + + final ReadableConfig tableOptions = helper.getOptions(); + + final DeliveryGuarantee deliveryGuarantee = validateDeprecatedSemantic(tableOptions); + validateTableSinkOptions(tableOptions); + + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); + + validatePKConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + valueEncodingFormat); + + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + + final Integer parallelism = tableOptions.getOptional(SINK_PARALLELISM).orElse(null); + + return createKafkaTableSink( + physicalDataType, + keyEncodingFormat.orElse(null), + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + getTopics(tableOptions), + getTopicPattern(tableOptions), + getKafkaProperties(context.getCatalogTable().getOptions()), + getFlinkKafkaPartitioner(tableOptions, context.getClassLoader()).orElse(null), + deliveryGuarantee, + parallelism, + tableOptions.get(TRANSACTIONAL_ID_PREFIX)); + } + + // -------------------------------------------------------------------------------------------- + + private static Optional>> getKeyDecodingFormat( + TableFactoryHelper helper) { + final Optional>> keyDecodingFormat = + helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, KEY_FORMAT); + keyDecodingFormat.ifPresent( + format -> { + if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "A key format should only deal with INSERT-only records. " + + "But %s has a changelog mode of %s.", + helper.getOptions().get(KEY_FORMAT), + format.getChangelogMode())); + } + }); + return keyDecodingFormat; + } + + private static Optional>> getKeyEncodingFormat( + TableFactoryHelper helper) { + final Optional>> keyEncodingFormat = + helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT); + keyEncodingFormat.ifPresent( + format -> { + if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "A key format should only deal with INSERT-only records. " + + "But %s has a changelog mode of %s.", + helper.getOptions().get(KEY_FORMAT), + format.getChangelogMode())); + } + }); + return keyEncodingFormat; + } + + private static DecodingFormat> getValueDecodingFormat( + TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> helper.discoverDecodingFormat( + DeserializationFormatFactory.class, VALUE_FORMAT)); + } + + private static EncodingFormat> getValueEncodingFormat( + TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat( + SerializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> helper.discoverEncodingFormat( + SerializationFormatFactory.class, VALUE_FORMAT)); + } + + private static void validatePKConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + Map options, + Format format) { + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + Configuration configuration = Configuration.fromMap(options); + String formatName = + configuration + .getOptional(FactoryUtil.FORMAT) + .orElse(configuration.get(VALUE_FORMAT)); + throw new ValidationException( + String.format( + "The Kafka table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), formatName)); + } + } + + private static DeliveryGuarantee validateDeprecatedSemantic(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_SEMANTIC).isPresent()) { + LOG.warn( + "{} is deprecated and will be removed. Please use {} instead.", + SINK_SEMANTIC.key(), + DELIVERY_GUARANTEE.key()); + return DeliveryGuarantee.valueOf( + tableOptions.get(SINK_SEMANTIC).toUpperCase().replace("-", "_")); + } + return tableOptions.get(DELIVERY_GUARANTEE); + } + + // -------------------------------------------------------------------------------------------- + + protected KafkaDynamicSource createKafkaTableSource( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + DecodingFormat> valueDecodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + StartupMode startupMode, + Map specificStartupOffsets, + long startupTimestampMillis, + BoundedMode boundedMode, + Map specificEndOffsets, + long endTimestampMillis, + String tableIdentifier) { + return new KafkaDynamicSource( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificEndOffsets, + endTimestampMillis, + false, + tableIdentifier); + } + + protected KafkaDynamicSink createKafkaTableSink( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + EncodingFormat> valueEncodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + KafkaPartitioner partitioner, + DeliveryGuarantee deliveryGuarantee, + Integer parallelism, + @Nullable String transactionalIdPrefix) { + return new KafkaDynamicSink( + physicalDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + partitioner, + deliveryGuarantee, + false, + SinkBufferFlushMode.DISABLED, + parallelism, + transactionalIdPrefix); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java new file mode 100644 index 00000000000..6a8493bfb22 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java @@ -0,0 +1,104 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.function.SerializableFunction; + +import java.io.IOException; +import java.util.Collection; + +/** + * A wrapper of a {@link Sink}. It will buffer the data emitted by the wrapper {@link SinkWriter} + * and only emit it when the buffer is full or a timer is triggered or a checkpoint happens. + * + *

The sink provides eventual consistency guarantees under {@link + * org.apache.flink.connector.base.DeliveryGuarantee#AT_LEAST_ONCE} because the updates are + * idempotent therefore duplicates have no effect. + */ +class ReducingUpsertSink + implements + TwoPhaseCommittingStatefulSink { + + private final TwoPhaseCommittingStatefulSink wrappedSink; + private final DataType physicalDataType; + private final int[] keyProjection; + private final SinkBufferFlushMode bufferFlushMode; + private final SerializableFunction valueCopyFunction; + + ReducingUpsertSink( + TwoPhaseCommittingStatefulSink wrappedSink, + DataType physicalDataType, + int[] keyProjection, + SinkBufferFlushMode bufferFlushMode, + SerializableFunction valueCopyFunction) { + this.wrappedSink = wrappedSink; + this.physicalDataType = physicalDataType; + this.keyProjection = keyProjection; + this.bufferFlushMode = bufferFlushMode; + this.valueCopyFunction = valueCopyFunction; + } + + @Override + public PrecommittingStatefulSinkWriter createWriter(InitContext context) + throws IOException { + return new ReducingUpsertWriter<>( + wrappedSink.createWriter(context), + physicalDataType, + keyProjection, + bufferFlushMode, + context.getProcessingTimeService(), + valueCopyFunction); + } + + @Override + public Committer createCommitter() throws IOException { + return wrappedSink.createCommitter(); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return wrappedSink.getCommittableSerializer(); + } + + @Override + public PrecommittingStatefulSinkWriter restoreWriter(InitContext context, + Collection recoveredState) + throws IOException { + final PrecommittingStatefulSinkWriter wrappedWriter = + wrappedSink.restoreWriter(context, recoveredState); + return new ReducingUpsertWriter<>( + wrappedWriter, + physicalDataType, + keyProjection, + bufferFlushMode, + context.getProcessingTimeService(), + valueCopyFunction); + } + + @Override + public SimpleVersionedSerializer getWriterStateSerializer() { + return wrappedSink.getWriterStateSerializer(); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java new file mode 100644 index 00000000000..00c332f6ac3 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java @@ -0,0 +1,190 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.apache.flink.types.RowKind.DELETE; +import static org.apache.flink.types.RowKind.UPDATE_AFTER; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.inlong.sort.kafka.table.DynamicKafkaRecordSerializationSchema.createProjectedRow; + +class ReducingUpsertWriter + implements + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter { + + private final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter wrappedWriter; + private final WrappedContext wrappedContext = new WrappedContext(); + private final int batchMaxRowNums; + private final Function valueCopyFunction; + private final Map> reduceBuffer = new HashMap<>(); + private final Function keyExtractor; + private final ProcessingTimeService timeService; + private final long batchIntervalMs; + + private boolean closed = false; + private long lastFlush = System.currentTimeMillis(); + + ReducingUpsertWriter( + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter wrappedWriter, + DataType physicalDataType, + int[] keyProjection, + SinkBufferFlushMode bufferFlushMode, + ProcessingTimeService timeService, + Function valueCopyFunction) { + checkArgument(bufferFlushMode != null && bufferFlushMode.isEnabled()); + this.wrappedWriter = checkNotNull(wrappedWriter); + this.timeService = checkNotNull(timeService); + this.batchMaxRowNums = bufferFlushMode.getBatchSize(); + this.batchIntervalMs = bufferFlushMode.getBatchIntervalMs(); + registerFlush(); + List fields = physicalDataType.getLogicalType().getChildren(); + final RowData.FieldGetter[] keyFieldGetters = + Arrays.stream(keyProjection) + .mapToObj( + targetField -> RowData.createFieldGetter( + fields.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + this.keyExtractor = rowData -> createProjectedRow(rowData, RowKind.INSERT, keyFieldGetters); + this.valueCopyFunction = valueCopyFunction; + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + wrappedContext.setContext(context); + addToBuffer(element, context.timestamp()); + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + sinkBuffer(); + wrappedWriter.flush(endOfInput); + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + return wrappedWriter.snapshotState(checkpointId); + } + + @Override + public void close() throws Exception { + if (!closed) { + closed = true; + wrappedWriter.close(); + } + } + + private void addToBuffer(RowData row, Long timestamp) throws IOException, InterruptedException { + RowData key = keyExtractor.apply(row); + RowData value = valueCopyFunction.apply(row); + reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp)); + + if (reduceBuffer.size() >= batchMaxRowNums) { + sinkBuffer(); + } + } + + private void registerFlush() { + if (closed) { + return; + } + timeService.registerTimer( + lastFlush + batchIntervalMs, + (t) -> { + if (t >= lastFlush + batchIntervalMs) { + sinkBuffer(); + } + registerFlush(); + }); + } + + private RowData changeFlag(RowData value) { + switch (value.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + value.setRowKind(UPDATE_AFTER); + break; + case UPDATE_BEFORE: + case DELETE: + value.setRowKind(DELETE); + } + return value; + } + + private void sinkBuffer() throws IOException, InterruptedException { + for (Tuple2 value : reduceBuffer.values()) { + wrappedContext.setTimestamp(value.f1); + wrappedWriter.write(value.f0, wrappedContext); + } + lastFlush = System.currentTimeMillis(); + reduceBuffer.clear(); + } + + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return wrappedWriter.prepareCommit(); + } + + /** + * Wrapper of {@link SinkWriter.Context}. + * + *

When records arrives, the {@link ReducingUpsertWriter} updates the current {@link + * SinkWriter.Context} and memorize the timestamp with the records. When flushing, the {@link + * ReducingUpsertWriter} will emit the records in the buffer with memorized timestamp. + */ + private static class WrappedContext implements SinkWriter.Context { + + private Long timestamp; + private SinkWriter.Context context; + + @Override + public long currentWatermark() { + checkNotNull(context, "context must be set before retrieving it."); + return context.currentWatermark(); + } + + @Override + public Long timestamp() { + return timestamp; + } + + public void setTimestamp(Long timestamp) { + this.timestamp = timestamp; + } + + public void setContext(SinkWriter.Context context) { + this.context = context; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java new file mode 100644 index 00000000000..580e6409324 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java @@ -0,0 +1,79 @@ +/* + * 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.inlong.sort.kafka.table; + +import java.io.Serializable; +import java.util.Objects; + +/** Sink buffer flush configuration. */ +public class SinkBufferFlushMode implements Serializable { + + private static final int DISABLED_BATCH_SIZE = 0; + private static final long DISABLED_BATCH_INTERVAL = 0L; + + public static final SinkBufferFlushMode DISABLED = + new SinkBufferFlushMode(DISABLED_BATCH_SIZE, DISABLED_BATCH_INTERVAL); + + private final int batchSize; + private final long batchIntervalMs; + + public SinkBufferFlushMode(int batchSize, long batchIntervalMs) { + this.batchSize = batchSize; + this.batchIntervalMs = batchIntervalMs; + + // validation + if (isEnabled() + && !(batchSize > DISABLED_BATCH_SIZE + && batchIntervalMs > DISABLED_BATCH_INTERVAL)) { + throw new IllegalArgumentException( + String.format( + "batchSize and batchInterval must greater than zero if buffer flush is enabled," + + " but got batchSize=%s and batchIntervalMs=%s", + batchSize, batchIntervalMs)); + } + } + + public int getBatchSize() { + return batchSize; + } + + public long getBatchIntervalMs() { + return batchIntervalMs; + } + + public boolean isEnabled() { + return !(batchSize == DISABLED_BATCH_SIZE && batchIntervalMs == DISABLED_BATCH_INTERVAL); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SinkBufferFlushMode that = (SinkBufferFlushMode) o; + return batchSize == that.batchSize && batchIntervalMs == that.batchIntervalMs; + } + + @Override + public int hashCode() { + return Objects.hash(batchSize, batchIntervalMs); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java new file mode 100644 index 00000000000..59e2cb5b0d2 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -0,0 +1,424 @@ +/* + * 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.inlong.sort.kafka.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.Format; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; +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.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopicPattern; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.getTopics; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.validateScanBoundedMode; + +/** Upsert-Kafka factory. */ +public class UpsertKafkaDynamicTableFactory + implements + DynamicTableSourceFactory, + DynamicTableSinkFactory { + + public static final String IDENTIFIER = "upsert-kafka"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(PROPS_BOOTSTRAP_SERVERS); + options.add(KEY_FORMAT); + options.add(VALUE_FORMAT); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(TOPIC); + options.add(TOPIC_PATTERN); + options.add(KEY_FIELDS_PREFIX); + options.add(VALUE_FIELDS_INCLUDE); + options.add(SINK_PARALLELISM); + options.add(SINK_BUFFER_FLUSH_INTERVAL); + options.add(SINK_BUFFER_FLUSH_MAX_ROWS); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); + return options; + } + + @Override + public Set> forwardOptions() { + return Stream.of(DELIVERY_GUARANTEE, TRANSACTIONAL_ID_PREFIX).collect(Collectors.toSet()); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + ReadableConfig tableOptions = helper.getOptions(); + DecodingFormat> keyDecodingFormat = + helper.discoverDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT); + DecodingFormat> valueDecodingFormat = + helper.discoverDecodingFormat(DeserializationFormatFactory.class, VALUE_FORMAT); + + // Validate the option data type. + helper.validateExcept(PROPERTIES_PREFIX); + validateSource( + tableOptions, + keyDecodingFormat, + valueDecodingFormat, + context.getPrimaryKeyIndexes()); + + Tuple2 keyValueProjections = + createKeyValueProjections(context.getCatalogTable()); + String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + // always use earliest to keep data integrity + StartupMode earliest = StartupMode.EARLIEST; + + final BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + + return new KafkaDynamicSource( + context.getPhysicalRowDataType(), + keyDecodingFormat, + new DecodingFormatWrapper(valueDecodingFormat), + keyValueProjections.f0, + keyValueProjections.f1, + keyPrefix, + getTopics(tableOptions), + getTopicPattern(tableOptions), + properties, + earliest, + Collections.emptyMap(), + 0, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, + true, + context.getObjectIdentifier().asSummaryString()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper( + this, autoCompleteSchemaRegistrySubject(context)); + + final ReadableConfig tableOptions = helper.getOptions(); + + EncodingFormat> keyEncodingFormat = + helper.discoverEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT); + EncodingFormat> valueEncodingFormat = + helper.discoverEncodingFormat(SerializationFormatFactory.class, VALUE_FORMAT); + + // Validate the option data type. + helper.validateExcept(PROPERTIES_PREFIX); + validateSink( + tableOptions, + keyEncodingFormat, + valueEncodingFormat, + context.getPrimaryKeyIndexes()); + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); + + Tuple2 keyValueProjections = + createKeyValueProjections(context.getCatalogTable()); + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + + Integer parallelism = tableOptions.get(SINK_PARALLELISM); + + int batchSize = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS); + Duration batchInterval = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL); + SinkBufferFlushMode flushMode = + new SinkBufferFlushMode(batchSize, batchInterval.toMillis()); + + // use {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. + // it will use hash partition if key is set else in round-robin behaviour. + return new KafkaDynamicSink( + context.getPhysicalRowDataType(), + context.getPhysicalRowDataType(), + keyEncodingFormat, + new EncodingFormatWrapper(valueEncodingFormat), + keyValueProjections.f0, + keyValueProjections.f1, + keyPrefix, + getTopics(tableOptions), + getTopicPattern(tableOptions), + properties, + null, + tableOptions.get(DELIVERY_GUARANTEE), + true, + flushMode, + parallelism, + tableOptions.get(TRANSACTIONAL_ID_PREFIX)); + } + + private Tuple2 createKeyValueProjections(ResolvedCatalogTable catalogTable) { + ResolvedSchema schema = catalogTable.getResolvedSchema(); + // primary key should validated earlier + List keyFields = schema.getPrimaryKey().get().getColumns(); + DataType physicalDataType = schema.toPhysicalRowDataType(); + + Configuration tableOptions = Configuration.fromMap(catalogTable.getOptions()); + // upsert-kafka will set key.fields to primary key fields by default + tableOptions.set(KEY_FIELDS, keyFields); + + int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + return Tuple2.of(keyProjection, valueProjection); + } + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + private static void validateSource( + ReadableConfig tableOptions, + Format keyFormat, + Format valueFormat, + int[] primaryKeyIndexes) { + validateScanBoundedMode(tableOptions); + validateFormat(keyFormat, valueFormat, tableOptions); + validatePKConstraints(primaryKeyIndexes); + } + + private static void validateSink( + ReadableConfig tableOptions, + Format keyFormat, + Format valueFormat, + int[] primaryKeyIndexes) { + validateFormat(keyFormat, valueFormat, tableOptions); + validatePKConstraints(primaryKeyIndexes); + validateSinkBufferFlush(tableOptions); + } + + private static void validateFormat( + Format keyFormat, Format valueFormat, ReadableConfig tableOptions) { + if (!keyFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { + String identifier = tableOptions.get(KEY_FORMAT); + throw new ValidationException( + String.format( + "'upsert-kafka' connector doesn't support '%s' as key format, " + + "because '%s' is not in insert-only mode.", + identifier, identifier)); + } + if (!valueFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { + String identifier = tableOptions.get(VALUE_FORMAT); + throw new ValidationException( + String.format( + "'upsert-kafka' connector doesn't support '%s' as value format, " + + "because '%s' is not in insert-only mode.", + identifier, identifier)); + } + } + + private static void validatePKConstraints(int[] schema) { + if (schema.length == 0) { + throw new ValidationException( + "'upsert-kafka' tables require to define a PRIMARY KEY constraint. " + + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. " + + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys."); + } + } + + private static void validateSinkBufferFlush(ReadableConfig tableOptions) { + int flushMaxRows = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS); + long flushIntervalMs = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis(); + if (flushMaxRows > 0 && flushIntervalMs > 0) { + // flush is enabled + return; + } + if (flushMaxRows <= 0 && flushIntervalMs <= 0) { + // flush is disabled + return; + } + // one of them is set which is not allowed + throw new ValidationException( + String.format( + "'%s' and '%s' must be set to be greater than zero together to enable sink buffer flushing.", + SINK_BUFFER_FLUSH_MAX_ROWS.key(), SINK_BUFFER_FLUSH_INTERVAL.key())); + } + + // -------------------------------------------------------------------------------------------- + // Format wrapper + // -------------------------------------------------------------------------------------------- + + /** + * It is used to wrap the decoding format and expose the desired changelog mode. It's only works + * for insert-only format. + */ + protected static class DecodingFormatWrapper + implements + DecodingFormat> { + + private final DecodingFormat> innerDecodingFormat; + + private static final ChangelogMode SOURCE_CHANGELOG_MODE = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + + public DecodingFormatWrapper( + DecodingFormat> innerDecodingFormat) { + this.innerDecodingFormat = innerDecodingFormat; + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + return innerDecodingFormat.createRuntimeDecoder(context, producedDataType); + } + + @Override + public ChangelogMode getChangelogMode() { + return SOURCE_CHANGELOG_MODE; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + DecodingFormatWrapper that = (DecodingFormatWrapper) obj; + return Objects.equals(innerDecodingFormat, that.innerDecodingFormat); + } + + @Override + public int hashCode() { + return Objects.hash(innerDecodingFormat); + } + } + + /** + * It is used to wrap the encoding format and expose the desired changelog mode. It's only works + * for insert-only format. + */ + protected static class EncodingFormatWrapper + implements + EncodingFormat> { + + private final EncodingFormat> innerEncodingFormat; + + public static final ChangelogMode SINK_CHANGELOG_MODE = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + + public EncodingFormatWrapper( + EncodingFormat> innerEncodingFormat) { + this.innerEncodingFormat = innerEncodingFormat; + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return innerEncodingFormat.createRuntimeEncoder(context, consumedDataType); + } + + @Override + public ChangelogMode getChangelogMode() { + return SINK_CHANGELOG_MODE; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + EncodingFormatWrapper that = (EncodingFormatWrapper) obj; + return Objects.equals(innerEncodingFormat, that.innerEncodingFormat); + } + + @Override + public int hashCode() { + return Objects.hash(innerEncodingFormat); + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/util/Preconditions.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/util/Preconditions.java new file mode 100644 index 00000000000..bc82ecd6eb9 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/util/Preconditions.java @@ -0,0 +1,310 @@ +/* + * 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.inlong.sort.kafka.util; + +import org.apache.flink.annotation.Internal; + +import javax.annotation.Nullable; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + * A collection of static utility methods to validate input. + * + *

This class is modelled after Google Guava's Preconditions class, and partly takes code from + * that class. We add this code to the Flink code base in order to reduce external dependencies. + */ +@Internal +public final class Preconditions { + + // ------------------------------------------------------------------------ + // Null checks + // ------------------------------------------------------------------------ + + /** + * Ensures that the given object reference is not null. Upon violation, a {@code + * NullPointerException} with no message is thrown. + * + * @param reference The object reference + * @return The object reference itself (generically typed). + * @throws NullPointerException Thrown, if the passed reference was null. + */ + public static T checkNotNull(@Nullable T reference) { + if (reference == null) { + throw new NullPointerException(); + } + return reference; + } + + /** + * Ensures that the given object reference is not null. Upon violation, a {@code + * NullPointerException} with the given message is thrown. + * + * @param reference The object reference + * @param errorMessage The message for the {@code NullPointerException} that is thrown if the + * check fails. + * @return The object reference itself (generically typed). + * @throws NullPointerException Thrown, if the passed reference was null. + */ + public static T checkNotNull(@Nullable T reference, @Nullable String errorMessage) { + if (reference == null) { + throw new NullPointerException(String.valueOf(errorMessage)); + } + return reference; + } + + /** + * Ensures that the given object reference is not null. Upon violation, a {@code + * NullPointerException} with the given message is thrown. + * + *

The error message is constructed from a template and an arguments array, after a similar + * fashion as {@link String#format(String, Object...)}, but supporting only {@code %s} as a + * placeholder. + * + * @param reference The object reference + * @param errorMessageTemplate The message template for the {@code NullPointerException} that is + * thrown if the check fails. The template substitutes its {@code %s} placeholders with the + * error message arguments. + * @param errorMessageArgs The arguments for the error message, to be inserted into the message + * template for the {@code %s} placeholders. + * @return The object reference itself (generically typed). + * @throws NullPointerException Thrown, if the passed reference was null. + */ + public static T checkNotNull( + T reference, + @Nullable String errorMessageTemplate, + @Nullable Object... errorMessageArgs) { + + if (reference == null) { + throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs)); + } + return reference; + } + + // ------------------------------------------------------------------------ + // Boolean Condition Checking (Argument) + // ------------------------------------------------------------------------ + + /** + * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if the + * condition is not met (evaluates to {@code false}). + * + * @param condition The condition to check + * @throws IllegalArgumentException Thrown, if the condition is violated. + */ + public static void checkArgument(boolean condition) { + if (!condition) { + throw new IllegalArgumentException(); + } + } + + /** + * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if the + * condition is not met (evaluates to {@code false}). The exception will have the given error + * message. + * + * @param condition The condition to check + * @param errorMessage The message for the {@code IllegalArgumentException} that is thrown if + * the check fails. + * @throws IllegalArgumentException Thrown, if the condition is violated. + */ + public static void checkArgument(boolean condition, @Nullable Object errorMessage) { + if (!condition) { + throw new IllegalArgumentException(String.valueOf(errorMessage)); + } + } + + /** + * Checks the given boolean condition, and throws an {@code IllegalArgumentException} if the + * condition is not met (evaluates to {@code false}). + * + * @param condition The condition to check + * @param errorMessageTemplate The message template for the {@code IllegalArgumentException} + * that is thrown if the check fails. The template substitutes its {@code %s} placeholders + * with the error message arguments. + * @param errorMessageArgs The arguments for the error message, to be inserted into the message + * template for the {@code %s} placeholders. + * @throws IllegalArgumentException Thrown, if the condition is violated. + */ + public static void checkArgument( + boolean condition, + @Nullable String errorMessageTemplate, + @Nullable Object... errorMessageArgs) { + + if (!condition) { + throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs)); + } + } + + // ------------------------------------------------------------------------ + // Boolean Condition Checking (State) + // ------------------------------------------------------------------------ + + /** + * Checks the given boolean condition, and throws an {@code IllegalStateException} if the + * condition is not met (evaluates to {@code false}). + * + * @param condition The condition to check + * @throws IllegalStateException Thrown, if the condition is violated. + */ + public static void checkState(boolean condition) { + if (!condition) { + throw new IllegalStateException(); + } + } + + /** + * Checks the given boolean condition, and throws an {@code IllegalStateException} if the + * condition is not met (evaluates to {@code false}). The exception will have the given error + * message. + * + * @param condition The condition to check + * @param errorMessage The message for the {@code IllegalStateException} that is thrown if the + * check fails. + * @throws IllegalStateException Thrown, if the condition is violated. + */ + public static void checkState(boolean condition, @Nullable Object errorMessage) { + if (!condition) { + throw new IllegalStateException(String.valueOf(errorMessage)); + } + } + + /** + * Checks the given boolean condition, and throws an {@code IllegalStateException} if the + * condition is not met (evaluates to {@code false}). + * + * @param condition The condition to check + * @param errorMessageTemplate The message template for the {@code IllegalStateException} that + * is thrown if the check fails. The template substitutes its {@code %s} placeholders with + * the error message arguments. + * @param errorMessageArgs The arguments for the error message, to be inserted into the message + * template for the {@code %s} placeholders. + * @throws IllegalStateException Thrown, if the condition is violated. + */ + public static void checkState( + boolean condition, + @Nullable String errorMessageTemplate, + @Nullable Object... errorMessageArgs) { + + if (!condition) { + throw new IllegalStateException(format(errorMessageTemplate, errorMessageArgs)); + } + } + + /** + * Ensures that the given index is valid for an array, list or string of the given size. + * + * @param index index to check + * @param size size of the array, list or string + * @throws IllegalArgumentException Thrown, if size is negative. + * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to + * size + */ + public static void checkElementIndex(int index, int size) { + checkArgument(size >= 0, "Size was negative."); + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException("Index: " + index + ", Size: " + size); + } + } + + /** + * Ensures that the given index is valid for an array, list or string of the given size. + * + * @param index index to check + * @param size size of the array, list or string + * @param errorMessage The message for the {@code IndexOutOfBoundsException} that is thrown if + * the check fails. + * @throws IllegalArgumentException Thrown, if size is negative. + * @throws IndexOutOfBoundsException Thrown, if the index negative or greater than or equal to + * size + */ + public static void checkElementIndex(int index, int size, @Nullable String errorMessage) { + checkArgument(size >= 0, "Size was negative."); + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException( + String.valueOf(errorMessage) + " Index: " + index + ", Size: " + size); + } + } + + /** + * Ensures that future has completed normally. + * + * @throws IllegalStateException Thrown, if future has not completed or it has completed + * exceptionally. + */ + public static void checkCompletedNormally(CompletableFuture future) { + checkState(future.isDone()); + if (future.isCompletedExceptionally()) { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + throw new IllegalStateException(e); + } + } + } + + // ------------------------------------------------------------------------ + // Utilities + // ------------------------------------------------------------------------ + + /** + * A simplified formatting method. Similar to {@link String#format(String, Object...)}, but with + * lower overhead (only String parameters, no locale, no format validation). + * + *

This method is taken quasi verbatim from the Guava Preconditions class. + */ + private static String format(@Nullable String template, @Nullable Object... args) { + final int numArgs = args == null ? 0 : args.length; + template = String.valueOf(template); // null -> "null" + + // start substituting the arguments into the '%s' placeholders + StringBuilder builder = new StringBuilder(template.length() + 16 * numArgs); + int templateStart = 0; + int i = 0; + while (i < numArgs) { + int placeholderStart = template.indexOf("%s", templateStart); + if (placeholderStart == -1) { + break; + } + builder.append(template.substring(templateStart, placeholderStart)); + builder.append(args[i++]); + templateStart = placeholderStart + 2; + } + builder.append(template.substring(templateStart)); + + // if we run out of placeholders, append the extra args in square braces + if (i < numArgs) { + builder.append(" ["); + builder.append(args[i++]); + while (i < numArgs) { + builder.append(", "); + builder.append(args[i++]); + } + builder.append(']'); + } + + return builder.toString(); + } + + // ------------------------------------------------------------------------ + + /** Private constructor to prevent instantiation. */ + private Preconditions() { + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000000..9b8bf8e042b --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,17 @@ +# 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. + +org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicTableFactory +org.apache.flink.streaming.connectors.kafka.table.UpsertKafkaDynamicTableFactory diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml index ce153aecea9..1cb1e7f0bae 100644 --- a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml @@ -36,6 +36,7 @@ elasticsearch-base elasticsearch6 elasticsearch7 + kafka