Skip to content
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][connectors-v2][kafka] Kafka supports custom schema #2371 #2783

Merged
merged 1 commit into from
Oct 19, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 24 additions & 5 deletions docs/en/connector-v2/source/kafka.md
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@ Source connector for Apache Kafka.
| commit_on_checkpoint | Boolean | no | true |
| kafka.* | String | no | - |
| common-options | | no | - |
| schema | | no | - |
| format | String | no | json |

### topic [string]

Expand Down Expand Up @@ -57,19 +59,36 @@ The way to specify parameters is to add the prefix `kafka.` to the original para

Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details.

### schema
The structure of the data, including field names and field types.

## format
Data format. The default format is json. Optional text format. The default field separator is ", ".
If you customize the delimiter, add the "field_delimiter" option.

## Example

### Simple

```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 = text
field_delimiter = "#“
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 @@ -64,4 +89,5 @@ public class Config {
* Determine the key of the kafka send partition
*/
public static final String PARTITION_KEY = "partition_key";

}
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,36 @@ 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 = TextDeserializationSchema.builder()
.seaTunnelRowType(typeInfo)
.delimiter(String.valueOf('\002'))
.build();
}
}
}
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,7 @@ 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}));
deserializationSchema.deserialize(record.value(), output);

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