forked from apache/inlong
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[INLONG-11494][Sort] Add Kafka connector on Flink 1.18
- Loading branch information
Showing
18 changed files
with
5,179 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
126 changes: 126 additions & 0 deletions
126
inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
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. | ||
--> | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
<parent> | ||
<groupId>org.apache.inlong</groupId> | ||
<artifactId>sort-connectors-v1.18</artifactId> | ||
<version>2.1.0-SNAPSHOT</version> | ||
</parent> | ||
|
||
<artifactId>sort-connector-kafka-v1.18</artifactId> | ||
<packaging>jar</packaging> | ||
<name>Apache InLong - Sort-connector-kafka</name> | ||
|
||
<properties> | ||
<inlong.root.dir>${project.parent.parent.parent.parent.parent.basedir}</inlong.root.dir> | ||
<flink.connector.kafka.version>3.2.0-1.18</flink.connector.kafka.version> | ||
<kafka.version>3.4.0</kafka.version> | ||
</properties> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.kafka</groupId> | ||
<artifactId>kafka-clients</artifactId> | ||
<version>${kafka.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.inlong</groupId> | ||
<artifactId>sort-common</artifactId> | ||
<version>${project.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.inlong</groupId> | ||
<artifactId>sort-connector-base</artifactId> | ||
<version>${project.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.flink</groupId> | ||
<artifactId>flink-connector-base</artifactId> | ||
<version>${flink.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.flink</groupId> | ||
<artifactId>flink-connector-kafka</artifactId> | ||
<version>${flink.connector.kafka.version}</version> | ||
</dependency> | ||
</dependencies> | ||
|
||
<build> | ||
<plugins> | ||
<plugin> | ||
<!-- Shade all the dependencies to avoid conflicts --> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-shade-plugin</artifactId> | ||
<executions> | ||
<execution> | ||
<id>shade-flink</id> | ||
<goals> | ||
<goal>shade</goal> | ||
</goals> | ||
<phase>package</phase> | ||
<configuration> | ||
<promoteTransitiveDependencies>true</promoteTransitiveDependencies> | ||
<artifactSet> | ||
<includes> | ||
<include>org.apache.inlong:*</include> | ||
<include>io.streamnative.connectors:kafka-flink-connector-origin*</include> | ||
<include>io.streamnative.connectors:flink-protobuf</include> | ||
<include>org.apache.kafka:*</include> | ||
<include>org.apache.flink:flink-connector-kafka</include> | ||
<include>com.google.protobuf:*</include> | ||
<include>org.bouncycastle*:*</include> | ||
<include>org.bouncycastle*:*</include> | ||
<include>javax.*:*</include> | ||
<include>org.lz4*:*</include> | ||
<include>org.slf4j:jul-to-slf4j</include> | ||
<include>io.airlift:*</include> | ||
</includes> | ||
</artifactSet> | ||
<filters> | ||
<filter> | ||
<artifact>org.apache.inlong:sort-connector-*</artifact> | ||
<includes> | ||
<include>org/apache/inlong/**</include> | ||
<include>META-INF/services/org.apache.flink.table.factories.Factory</include> | ||
</includes> | ||
</filter> | ||
<filter> | ||
<artifact>*:*</artifact> | ||
<excludes> | ||
<exclude>log4j.properties</exclude> | ||
<exclude>META-INF/*.SF</exclude> | ||
<exclude>META-INF/*.DSA</exclude> | ||
<exclude>META-INF/*.RSA</exclude> | ||
</excludes> | ||
</filter> | ||
</filters> | ||
<transformers> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" /> | ||
<transformer implementation="org.apache.maven.plugins.shade.resource.PluginXmlResourceTransformer" /> | ||
</transformers> | ||
</configuration> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
</project> |
258 changes: 258 additions & 0 deletions
258
.../sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,258 @@ | ||
/* | ||
* 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 <code> | ||
* String</code> type. | ||
* | ||
* <pre>{@code | ||
* KafkaSource<String> source = KafkaSource | ||
* .<String>builder() | ||
* .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) | ||
* .setGroupId("MyGroup") | ||
* .setTopics(Arrays.asList(TOPIC1, TOPIC2)) | ||
* .setDeserializer(new TestingKafkaRecordDeserializationSchema()) | ||
* .setStartingOffsets(OffsetsInitializer.earliest()) | ||
* .build(); | ||
* }</pre> | ||
* | ||
* <p>{@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator} only supports | ||
* adding new splits and not removing splits in split discovery. | ||
* | ||
* <p>See {@link KafkaSourceBuilder} for more details on how to configure this source. | ||
* | ||
* @param <OUT> the output type of the source. | ||
* copied from org.apache.flink:flink-connector-kafka:1.18.0 | ||
*/ | ||
// TODO: Add a variable metricSchema to report audit information | ||
@PublicEvolving | ||
public class KafkaSource<OUT> | ||
implements | ||
Source<OUT, KafkaPartitionSplit, KafkaSourceEnumState>, | ||
ResultTypeQueryable<OUT> { | ||
|
||
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<OUT> deserializationSchema; | ||
// The configurations. | ||
private final Properties props; | ||
// Client rackId callback | ||
private final SerializableSupplier<String> rackIdSupplier; | ||
|
||
KafkaSource( | ||
KafkaSubscriber subscriber, | ||
OffsetsInitializer startingOffsetsInitializer, | ||
@Nullable OffsetsInitializer stoppingOffsetsInitializer, | ||
Boundedness boundedness, | ||
KafkaRecordDeserializationSchema<OUT> deserializationSchema, | ||
Properties props, | ||
SerializableSupplier<String> 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 <OUT> KafkaSourceBuilder<OUT> builder() { | ||
return new KafkaSourceBuilder<>(); | ||
} | ||
|
||
@Override | ||
public Boundedness getBoundedness() { | ||
return this.boundedness; | ||
} | ||
|
||
@Internal | ||
@Override | ||
public SourceReader<OUT, KafkaPartitionSplit> createReader(SourceReaderContext readerContext) | ||
throws Exception { | ||
return createReader(readerContext, (ignore) -> { | ||
}); | ||
} | ||
|
||
@VisibleForTesting | ||
SourceReader<OUT, KafkaPartitionSplit> createReader( | ||
SourceReaderContext readerContext, Consumer<Collection<String>> splitFinishedHook) | ||
throws Exception { | ||
FutureCompletingBlockingQueue<RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>>> 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<KafkaPartitionSplitReader> splitReaderSupplier = | ||
() -> new KafkaPartitionSplitReader( | ||
props, | ||
readerContext, | ||
kafkaSourceReaderMetrics, | ||
Optional.ofNullable(rackIdSupplier) | ||
.map(Supplier::get) | ||
.orElse(null)); | ||
KafkaRecordEmitter<OUT> recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); | ||
|
||
return new KafkaSourceReader<>( | ||
elementsQueue, | ||
new KafkaSourceFetcherManager( | ||
elementsQueue, splitReaderSupplier::get, splitFinishedHook), | ||
recordEmitter, | ||
toConfiguration(props), | ||
readerContext, | ||
kafkaSourceReaderMetrics); | ||
} | ||
|
||
@Internal | ||
@Override | ||
public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> createEnumerator( | ||
SplitEnumeratorContext<KafkaPartitionSplit> enumContext) { | ||
return new KafkaSourceEnumerator( | ||
subscriber, | ||
startingOffsetsInitializer, | ||
stoppingOffsetsInitializer, | ||
props, | ||
enumContext, | ||
boundedness); | ||
} | ||
|
||
@Internal | ||
@Override | ||
public SplitEnumerator<KafkaPartitionSplit, KafkaSourceEnumState> restoreEnumerator( | ||
SplitEnumeratorContext<KafkaPartitionSplit> enumContext, | ||
KafkaSourceEnumState checkpoint) | ||
throws IOException { | ||
return new KafkaSourceEnumerator( | ||
subscriber, | ||
startingOffsetsInitializer, | ||
stoppingOffsetsInitializer, | ||
props, | ||
enumContext, | ||
boundedness, | ||
checkpoint); | ||
} | ||
|
||
@Internal | ||
@Override | ||
public SimpleVersionedSerializer<KafkaPartitionSplit> getSplitSerializer() { | ||
return new KafkaPartitionSplitSerializer(); | ||
} | ||
|
||
@Internal | ||
@Override | ||
public SimpleVersionedSerializer<KafkaSourceEnumState> getEnumeratorCheckpointSerializer() { | ||
return new KafkaSourceEnumStateSerializer(); | ||
} | ||
|
||
@Override | ||
public TypeInformation<OUT> 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; | ||
} | ||
} |
Oops, something went wrong.