TyrantLucifer commented on code in PR #3742:
URL:
https://github.com/apache/incubator-seatunnel/pull/3742#discussion_r1082086864
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -61,10 +64,13 @@ public class KafkaSinkWriter implements
SinkWriter<SeaTunnelRow, KafkaCommitInfo
private final SinkWriter.Context context;
private String transactionPrefix;
+ private String topic;
private long lastCheckpointId = 0;
+ private boolean isExtractTopic;
private final KafkaProduceSender<byte[], byte[]> kafkaProducerSender;
private final SeaTunnelRowSerializer<byte[], byte[]>
seaTunnelRowSerializer;
+ private final Function<SeaTunnelRow, String> topicExtractor;
Review Comment:
Remove it. This may affect performance.
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -74,6 +80,9 @@ public KafkaSinkWriter(
Config pluginConfig,
List<KafkaSinkState> kafkaStates) {
this.context = context;
+ this.topic = pluginConfig.getString(TOPIC.key());
+ this.isExtractTopic = setExtractTopic(this.topic);
Review Comment:
```suggestion
Pair<Boolean, String> topicResult = isExtractTopic(this.topic);
this.isExtractTopic = topicResult.getLetf();
this.topic = topicResult.getRight();
```
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -102,7 +111,10 @@ public KafkaSinkWriter(
@Override
public void write(SeaTunnelRow element) {
- ProducerRecord<byte[], byte[]> producerRecord =
seaTunnelRowSerializer.serializeRow(element);
+ if (isExtractTopic) {
+ topic = topicExtractor.apply(element);
+ }
+ ProducerRecord<byte[], byte[]> producerRecord =
seaTunnelRowSerializer.serializeRow(topic, element);
Review Comment:
```suggestion
ProducerRecord<byte[], byte[]> producerRecord =
seaTunnelRowSerializer.serializeRow(extractTopic(element), element);
```
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -199,4 +211,38 @@ private List<String> getPartitionKeyFields(Config
pluginConfig, SeaTunnelRowType
}
return Collections.emptyList();
}
+
+ private boolean setExtractTopic(String topicConfig){
+ String regex = "\\$\\{(.*?)\\}";
+ Pattern pattern = Pattern.compile(regex, Pattern.DOTALL);
+ Matcher matcher = pattern.matcher(topicConfig);
+ if (matcher.find()) {
+ return true;
+ }
+ return false;
+ }
+
+ private Function<SeaTunnelRow, String> createTopicExtractor(String
topicConfig, SeaTunnelRowType seaTunnelRowType) {
+ String regex = "\\$\\{(.*?)\\}";
+ Pattern pattern = Pattern.compile(regex, Pattern.DOTALL);
+ Matcher matcher = pattern.matcher(topicConfig);
+ if (!matcher.find()) {
+ return row -> topicConfig;
+ }
+ String topicField = matcher.group(1);
+ List<String> fieldNames =
Arrays.asList(seaTunnelRowType.getFieldNames());
+ if (!fieldNames.contains(topicField)) {
+ throw new KafkaConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT,
+ String.format("Field name { %s } is not found!",
topicField));
+ }
+ int topicFieldIndex = seaTunnelRowType.indexOf(topicField);
+ return row -> {
+ Object topicFieldValue = row.getField(topicFieldIndex);
+ if (topicFieldValue == null) {
+ throw new
KafkaConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT,
+ "The column value is empty!");
+ }
+ return topicFieldValue.toString();
+ };
+ }
Review Comment:
```suggestion
private String extractTopic(SeaTunnelRowType seaTunnelRowType) {
if (!isExtractTopic) {
return topic;
}
List<String> fieldNames =
Arrays.asList(seaTunnelRowType.getFieldNames());
if (!fieldNames.contains(topic)) {
throw new
KafkaConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT,
String.format("Field name { %s } is not found!", topic));
}
int topicFieldIndex = seaTunnelRowType.indexOf(topic);
Object topicFieldValue = row.getField(topicFieldIndex);
if (topicFieldValue == null) {
throw new
KafkaConnectorException(CommonErrorCode.ILLEGAL_ARGUMENT,
"The column value is empty!");
}
return topicFieldValue.toString();
}
```
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -74,6 +80,9 @@ public KafkaSinkWriter(
Config pluginConfig,
List<KafkaSinkState> kafkaStates) {
this.context = context;
+ this.topic = pluginConfig.getString(TOPIC.key());
+ this.isExtractTopic = setExtractTopic(this.topic);
+ this.topicExtractor = createTopicExtractor(this.topic,
seaTunnelRowType);
Review Comment:
Remove it.
##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -199,4 +211,38 @@ private List<String> getPartitionKeyFields(Config
pluginConfig, SeaTunnelRowType
}
return Collections.emptyList();
}
+
+ private boolean setExtractTopic(String topicConfig){
+ String regex = "\\$\\{(.*?)\\}";
+ Pattern pattern = Pattern.compile(regex, Pattern.DOTALL);
+ Matcher matcher = pattern.matcher(topicConfig);
+ if (matcher.find()) {
+ return true;
+ }
+ return false;
+ }
Review Comment:
```suggestion
private Pair<Boolean, String> isExtractTopic(String topicConfig){
String regex = "\\$\\{(.*?)\\}";
Pattern pattern = Pattern.compile(regex, Pattern.DOTALL);
Matcher matcher = pattern.matcher(topicConfig);
if (matcher.find()) {
return Pair.of(true, match.group(1));
}
return Pair.of(false, topicConfig);
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]