-
Notifications
You must be signed in to change notification settings - Fork 1.8k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Feature][seatunnel-connectors-v2][connector-kafka] Kafka supports custom schema #2371 #2720
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,14 +17,15 @@ | |
|
||
package org.apache.seatunnel.connectors.seatunnel.kafka.source; | ||
|
||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVER; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.COMMIT_ON_CHECKPOINT; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONSUMER_GROUP; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig.BOOTSTRAP_SERVER; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig.COMMIT_ON_CHECKPOINT; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig.CONSUMER_GROUP; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig.PATTERN; | ||
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig.TOPIC; | ||
|
||
import org.apache.seatunnel.api.common.PrepareFailException; | ||
import org.apache.seatunnel.api.common.SeaTunnelContext; | ||
import org.apache.seatunnel.api.serialization.DeserializationSchema; | ||
import org.apache.seatunnel.api.source.Boundedness; | ||
import org.apache.seatunnel.api.source.SeaTunnelSource; | ||
import org.apache.seatunnel.api.source.SourceReader; | ||
|
@@ -38,18 +39,21 @@ | |
import org.apache.seatunnel.common.config.TypesafeConfigUtils; | ||
import org.apache.seatunnel.common.constants.JobMode; | ||
import org.apache.seatunnel.common.constants.PluginType; | ||
import org.apache.seatunnel.connectors.seatunnel.common.schema.SeaTunnelSchema; | ||
import org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaConfig; | ||
import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; | ||
|
||
import org.apache.seatunnel.format.json.JsonDeserializationSchema; | ||
import org.apache.seatunnel.shade.com.typesafe.config.Config; | ||
|
||
import com.google.auto.service.AutoService; | ||
|
||
import java.util.Properties; | ||
|
||
@AutoService(SeaTunnelSource.class) | ||
public class KafkaSource implements SeaTunnelSource<SeaTunnelRow, KafkaSourceSplit, KafkaSourceState> { | ||
|
||
private static final String DEFAULT_CONSUMER_GROUP = "SeaTunnel-Consumer-Group"; | ||
public class KafkaSource<T> implements SeaTunnelSource<T, KafkaSourceSplit, KafkaSourceState> { | ||
public static final String IDENTIFIER = "Kafka"; | ||
private DeserializationSchema<T> deserializationSchema; | ||
|
||
private final ConsumerMetadata metadata = new ConsumerMetadata(); | ||
private SeaTunnelRowType typeInfo; | ||
|
@@ -62,50 +66,47 @@ public Boundedness getBoundedness() { | |
|
||
@Override | ||
public String getPluginName() { | ||
return "Kafka"; | ||
return IDENTIFIER; | ||
} | ||
|
||
@Override | ||
public void prepare(Config config) throws PrepareFailException { | ||
CheckResult result = CheckConfigUtil.checkAllExists(config, TOPIC, BOOTSTRAP_SERVER); | ||
public void prepare(Config pluginConfig) throws PrepareFailException { | ||
CheckResult result = CheckConfigUtil.checkAllExists(pluginConfig, TOPIC, BOOTSTRAP_SERVER); | ||
if (!result.isSuccess()) { | ||
throw new PrepareFailException(getPluginName(), PluginType.SOURCE, result.getMsg()); | ||
} | ||
this.metadata.setTopic(config.getString(TOPIC)); | ||
if (config.hasPath(PATTERN)) { | ||
this.metadata.setPattern(config.getBoolean(PATTERN)); | ||
this.metadata.setTopic(pluginConfig.getString(TOPIC)); | ||
if (pluginConfig.hasPath(PATTERN)) { | ||
this.metadata.setPattern(pluginConfig.getBoolean(PATTERN)); | ||
} | ||
this.metadata.setBootstrapServer(config.getString(BOOTSTRAP_SERVER)); | ||
this.metadata.setBootstrapServer(pluginConfig.getString(BOOTSTRAP_SERVER)); | ||
this.metadata.setProperties(new Properties()); | ||
|
||
if (config.hasPath(CONSUMER_GROUP)) { | ||
this.metadata.setConsumerGroup(config.getString(CONSUMER_GROUP)); | ||
if (pluginConfig.hasPath(CONSUMER_GROUP)) { | ||
this.metadata.setConsumerGroup(pluginConfig.getString(CONSUMER_GROUP)); | ||
} else { | ||
this.metadata.setConsumerGroup(DEFAULT_CONSUMER_GROUP); | ||
this.metadata.setConsumerGroup(KafkaConfig.DEFAULT_CONSUMER_GROUP); | ||
} | ||
|
||
if (config.hasPath(COMMIT_ON_CHECKPOINT)) { | ||
this.metadata.setCommitOnCheckpoint(config.getBoolean(COMMIT_ON_CHECKPOINT)); | ||
if (pluginConfig.hasPath(COMMIT_ON_CHECKPOINT)) { | ||
this.metadata.setCommitOnCheckpoint(pluginConfig.getBoolean(COMMIT_ON_CHECKPOINT)); | ||
} | ||
|
||
TypesafeConfigUtils.extractSubConfig(config, "kafka.", false).entrySet().forEach(e -> { | ||
TypesafeConfigUtils.extractSubConfig(pluginConfig, "kafka.", false).entrySet().forEach(e -> { | ||
this.metadata.getProperties().put(e.getKey(), String.valueOf(e.getValue().unwrapped())); | ||
}); | ||
|
||
// TODO support user custom row type | ||
this.typeInfo = new SeaTunnelRowType(new String[]{"topic", "raw_message"}, | ||
new SeaTunnelDataType[]{BasicType.STRING_TYPE, BasicType.STRING_TYPE}); | ||
|
||
setDeserialization(pluginConfig); | ||
} | ||
|
||
@Override | ||
public SeaTunnelRowType getProducedType() { | ||
return this.typeInfo; | ||
public SeaTunnelDataType<T> getProducedType() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The same as above. |
||
return deserializationSchema.getProducedType(); | ||
} | ||
|
||
@Override | ||
public SourceReader<SeaTunnelRow, KafkaSourceSplit> createReader(SourceReader.Context readerContext) throws Exception { | ||
return new KafkaSourceReader(this.metadata, this.typeInfo, readerContext); | ||
public SourceReader<T, KafkaSourceSplit> createReader(SourceReader.Context readerContext) throws Exception { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The same as above. |
||
return new KafkaSourceReader<>(this.metadata, this.deserializationSchema, readerContext); | ||
} | ||
|
||
@Override | ||
|
@@ -122,4 +123,15 @@ public SourceSplitEnumerator<KafkaSourceSplit, KafkaSourceState> restoreEnumerat | |
public void setSeaTunnelContext(SeaTunnelContext seaTunnelContext) { | ||
this.seaTunnelContext = seaTunnelContext; | ||
} | ||
|
||
private void setDeserialization(Config pluginConfig) { | ||
SeaTunnelRowType rowType; | ||
if (pluginConfig.hasPath(KafkaConfig.SCHEMA)) { | ||
Config schema = pluginConfig.getConfig(KafkaConfig.SCHEMA); | ||
rowType = SeaTunnelSchema.buildWithConfig(schema).getSeaTunnelRowType(); | ||
} else { | ||
rowType = SeaTunnelSchema.buildSimpleTextSchema(); | ||
} | ||
deserializationSchema = (DeserializationSchema<T>) new JsonDeserializationSchema(false, false, rowType); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
package org.apache.seatunnel.connectors.seatunnel.kafka.source; | ||
|
||
import org.apache.seatunnel.api.serialization.DeserializationSchema; | ||
import org.apache.seatunnel.api.source.Boundedness; | ||
import org.apache.seatunnel.api.source.Collector; | ||
import org.apache.seatunnel.api.source.SourceReader; | ||
|
@@ -46,7 +47,7 @@ | |
import java.util.Set; | ||
import java.util.stream.Collectors; | ||
|
||
public class KafkaSourceReader implements SourceReader<SeaTunnelRow, KafkaSourceSplit> { | ||
public class KafkaSourceReader<T> implements SourceReader<T, KafkaSourceSplit> { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The same as above. |
||
|
||
private static final long THREAD_WAIT_TIME = 500L; | ||
private static final long POLL_TIMEOUT = 10000L; | ||
|
@@ -58,15 +59,16 @@ public class KafkaSourceReader implements SourceReader<SeaTunnelRow, KafkaSource | |
private final ConsumerMetadata metadata; | ||
private final Set<KafkaSourceSplit> sourceSplits; | ||
private final Map<TopicPartition, Long> endOffset; | ||
// TODO support user custom type | ||
private SeaTunnelRowType typeInfo; | ||
|
||
private final DeserializationSchema<T> deserializationSchema; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The same as above. |
||
private volatile boolean isRunning; | ||
|
||
KafkaSourceReader(ConsumerMetadata metadata, SeaTunnelRowType typeInfo, | ||
KafkaSourceReader(ConsumerMetadata metadata, | ||
DeserializationSchema<T> deserializationSchema, | ||
SourceReader.Context context) { | ||
this.metadata = metadata; | ||
this.context = context; | ||
this.typeInfo = typeInfo; | ||
this.deserializationSchema = deserializationSchema; | ||
this.sourceSplits = new HashSet<>(); | ||
this.endOffset = new HashMap<>(); | ||
} | ||
|
@@ -87,7 +89,7 @@ public void close() throws IOException { | |
} | ||
|
||
@Override | ||
public void pollNext(Collector<SeaTunnelRow> output) throws Exception { | ||
public void pollNext(Collector<T> output) throws Exception { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The same as above. |
||
if (sourceSplits.isEmpty() || sourceSplits.size() != this.endOffset.size()) { | ||
Thread.sleep(THREAD_WAIT_TIME); | ||
return; | ||
|
@@ -101,9 +103,13 @@ public void pollNext(Collector<SeaTunnelRow> output) throws Exception { | |
for (TopicPartition partition : partitions) { | ||
for (ConsumerRecord<byte[], byte[]> record : records.records(partition)) { | ||
|
||
String v = stringDeserializer.deserialize(partition.topic(), record.value()); | ||
String t = partition.topic(); | ||
output.collect(new SeaTunnelRow(new Object[]{t, v})); | ||
if(deserializationSchema != null){ | ||
deserializationSchema.deserialize(record.value(),output); | ||
}else{ | ||
String v = stringDeserializer.deserialize(partition.topic(), record.value()); | ||
String t = partition.topic(); | ||
output.collect((T) new SeaTunnelRow(new Object[]{t, v})); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If user does not assign schema, it will generate a simple schema as the following shown:
So the row only have one field named |
||
} | ||
|
||
if (Boundedness.BOUNDED.equals(context.getBoundedness()) && | ||
record.offset() >= endOffset.get(partition)) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
<?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"> | ||
<parent> | ||
<groupId>org.apache.seatunnel</groupId> | ||
<artifactId>seatunnel-flink-connector-v2-e2e</artifactId> | ||
<version>${revision}</version> | ||
</parent> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<artifactId>connector-kafka-flink-e2e</artifactId> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.seatunnel</groupId> | ||
<artifactId>connector-flink-e2e-base</artifactId> | ||
<version>${project.version}</version> | ||
<classifier>tests</classifier> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
|
||
<!-- SeaTunnel connectors --> | ||
<dependency> | ||
<groupId>org.apache.seatunnel</groupId> | ||
<artifactId>connector-kafka</artifactId> | ||
<version>${project.version}</version> | ||
<scope>test</scope> | ||
</dependency> | ||
</dependencies> | ||
|
||
</project> |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,20 @@ | ||
package org.apache.seatunnel.e2e.flink.v2.kafka; | ||
|
||
import org.apache.seatunnel.e2e.flink.FlinkContainer; | ||
import org.junit.jupiter.api.Assertions; | ||
import org.junit.jupiter.api.Test; | ||
import org.testcontainers.containers.Container; | ||
|
||
import java.io.IOException; | ||
|
||
public class KafkaIT extends FlinkContainer { | ||
|
||
/** | ||
* kafka source -> console sink | ||
*/ | ||
@Test | ||
public void testFakeSourceToLocalFileText() throws IOException, InterruptedException { | ||
Container.ExecResult execResult = executeSeaTunnelFlinkJob("/kafka/kafka_to_console.conf"); | ||
Assertions.assertEquals(0, execResult.getExitCode()); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
# | ||
# 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. | ||
# | ||
###### | ||
###### This config file is a demonstration of streaming processing in seatunnel config | ||
###### | ||
|
||
env { | ||
# You can set flink configuration here | ||
execution.parallelism = 1 | ||
#execution.checkpoint.interval = 10000 | ||
#execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" | ||
} | ||
|
||
source { | ||
Kafka { | ||
result_table_name = "kafka" | ||
schema = { | ||
fields { | ||
name = "string" | ||
age = "int" | ||
Comment on lines
+33
to
+34
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. test all datatypes? |
||
} | ||
} | ||
#format = text // or json | ||
#text.column.delimiter = "," // example | ' ' @ _ | ||
topic = "test_csv" | ||
bootstrap.server = "hadoop101:9092,hadoop102:9092,hadoop103:9092" | ||
} | ||
|
||
# If you would like to get more information about how to configure seatunnel and see full list of source plugins, | ||
# please go to https://seatunnel.apache.org/docs/connector-v2/source/KafkaSource | ||
} | ||
|
||
transform { | ||
sql { | ||
sql = "select name,age from kafka" | ||
} | ||
|
||
# If you would like to get more information about how to configure seatunnel and see full list of transform plugins, | ||
# please go to https://seatunnel.apache.org/docs/transform/sql | ||
} | ||
|
||
sink { | ||
Console {} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why change the generic type?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Refer to the PULSAR implementation