Skip to content

Commit

Permalink
[Feature][seatunnel-connectors-v2][connector-kafka] Kafka supports cu…
Browse files Browse the repository at this point in the history
…stom schema
  • Loading branch information
eyys committed Oct 10, 2022
1 parent c9dc7c5 commit 0a8c1b3
Show file tree
Hide file tree
Showing 21 changed files with 1,580 additions and 21 deletions.
19 changes: 14 additions & 5 deletions docs/en/connector-v2/source/kafka.md
Original file line number Diff line number Diff line change
Expand Up @@ -64,12 +64,21 @@ Source plugin common parameters, please refer to [Source Common Options](common-
```hocon
source {
Kafka {
topic = "seatunnel"
bootstrap.servers = "localhost:9092"
consumer.group = "seatunnel_group"
Kafka {
result_table_name = "kafka_name"
schema = {
fields {
name = "string"
age = "int"
}
}
format = json
topic = "topic_1,topic_2,topic_3"
bootstrap.server = "localhost:9092"
kafka.max.poll.records = 500
kafka.client.id = client_1
}
}
```

Expand Down
12 changes: 11 additions & 1 deletion seatunnel-connectors-v2/connector-kafka/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -28,14 +28,19 @@
<modelVersion>4.0.0</modelVersion>

<artifactId>connector-kafka</artifactId>

<properties>
<kafka.client.version>3.2.0</kafka.client.version>
</properties>

<dependencies>

<!-- TODO add to dependency management after version unify-->
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>connector-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
Expand All @@ -46,6 +51,11 @@
<artifactId>seatunnel-format-json</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>seatunnel-format-text</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>

</project>
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,31 @@ public class Config {
*/
public static final String TRANSACTION_PREFIX = "transaction_prefix";

/**
* User-defined schema
*/
public static final String SCHEMA = "schema";

/**
* data format
*/
public static final String FORMAT = "format";

/**
* The default data format is JSON
*/
public static final String DEFAULT_FORMAT = "json";

/**
* field delimiter
*/
public static final String FIELD_DELIMITER = "field_delimiter";

/**
* The default field delimiter is “,”
*/
public static final String DEFAULT_FIELD_DELIMITER = ",";

/**
* Send information according to the specified partition.
*/
Expand All @@ -59,4 +84,5 @@ public class Config {
* Determine the partition to send based on the content of the message.
*/
public static final String ASSIGN_PARTITIONS = "assign_partitions";

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,25 +20,32 @@
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVERS;
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.DEFAULT_FIELD_DELIMITER;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.DEFAULT_FORMAT;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FIELD_DELIMITER;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FORMAT;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.PATTERN;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.SCHEMA;
import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.TOPIC;

import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.common.PrepareFailException;
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;
import org.apache.seatunnel.api.source.SourceSplitEnumerator;
import org.apache.seatunnel.api.table.type.BasicType;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.config.CheckConfigUtil;
import org.apache.seatunnel.common.config.CheckResult;
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.state.KafkaSourceState;
import org.apache.seatunnel.format.json.JsonDeserializationSchema;
import org.apache.seatunnel.format.text.TextDeserializationSchema;

import org.apache.seatunnel.shade.com.typesafe.config.Config;

Expand All @@ -52,6 +59,7 @@ public class KafkaSource implements SeaTunnelSource<SeaTunnelRow, KafkaSourceSpl
private static final String DEFAULT_CONSUMER_GROUP = "SeaTunnel-Consumer-Group";

private final ConsumerMetadata metadata = new ConsumerMetadata();
private DeserializationSchema<SeaTunnelRow> deserializationSchema;
private SeaTunnelRowType typeInfo;
private JobContext jobContext;

Expand Down Expand Up @@ -92,10 +100,7 @@ public void prepare(Config config) throws PrepareFailException {
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(config);
}

@Override
Expand All @@ -105,7 +110,7 @@ public SeaTunnelRowType getProducedType() {

@Override
public SourceReader<SeaTunnelRow, KafkaSourceSplit> createReader(SourceReader.Context readerContext) throws Exception {
return new KafkaSourceReader(this.metadata, this.typeInfo, readerContext);
return new KafkaSourceReader(this.metadata, deserializationSchema, readerContext);
}

@Override
Expand All @@ -122,4 +127,33 @@ public SourceSplitEnumerator<KafkaSourceSplit, KafkaSourceState> restoreEnumerat
public void setJobContext(JobContext jobContext) {
this.jobContext = jobContext;
}

private void setDeserialization(Config config) {
if (config.hasPath(SCHEMA)) {
Config schema = config.getConfig(SCHEMA);
typeInfo = SeaTunnelSchema.buildWithConfig(schema).getSeaTunnelRowType();
String format = DEFAULT_FORMAT;
if (config.hasPath(FORMAT)) {
format = config.getString(FORMAT);
}
if (DEFAULT_FORMAT.equals(format)) {
deserializationSchema = new JsonDeserializationSchema(false, false, typeInfo);
} else if ("text".equals(format)) {
String delimiter = DEFAULT_FIELD_DELIMITER;
if (config.hasPath(FIELD_DELIMITER)) {
delimiter = config.getString(FIELD_DELIMITER);
}
deserializationSchema = TextDeserializationSchema.builder()
.seaTunnelRowType(typeInfo)
.delimiter(delimiter)
.build();
} else {
// TODO: use format SPI
throw new UnsupportedOperationException("Unsupported format: " + format);
}
} else {
typeInfo = SeaTunnelSchema.buildSimpleTextSchema();
this.deserializationSchema = null;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,11 @@

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;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;

import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
Expand Down Expand Up @@ -59,15 +59,15 @@ public class KafkaSourceReader implements SourceReader<SeaTunnelRow, KafkaSource
private final ConcurrentMap<TopicPartition, KafkaSourceSplit> sourceSplitMap;
private final Map<TopicPartition, KafkaConsumerThread> consumerThreadMap;
private final ExecutorService executorService;
// TODO support user custom type
private SeaTunnelRowType typeInfo;
private final DeserializationSchema<SeaTunnelRow> deserializationSchema;

KafkaSourceReader(ConsumerMetadata metadata, SeaTunnelRowType typeInfo,
KafkaSourceReader(ConsumerMetadata metadata,
DeserializationSchema<SeaTunnelRow> deserializationSchema,
SourceReader.Context context) {
this.metadata = metadata;
this.context = context;
this.typeInfo = typeInfo;
this.sourceSplits = new HashSet<>();
this.deserializationSchema = deserializationSchema;
this.consumerThreadMap = new ConcurrentHashMap<>();
this.sourceSplitMap = new ConcurrentHashMap<>();
this.checkpointOffsetMap = new ConcurrentHashMap<>();
Expand Down Expand Up @@ -114,9 +114,12 @@ public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
List<ConsumerRecord<byte[], byte[]>> recordList = records.records(partition);
for (ConsumerRecord<byte[], byte[]> record : recordList) {

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 content = stringDeserializer.deserialize(partition.topic(), record.value());
output.collect(new SeaTunnelRow(new Object[]{content}));
}

if (Boundedness.BOUNDED.equals(context.getBoundedness()) &&
record.offset() >= sourceSplit.getEndOffset()) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
<?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>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>connector-console</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.seatunnel</groupId>
<artifactId>connector-assert</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

</project>
Loading

0 comments on commit 0a8c1b3

Please sign in to comment.