hailin0 commented on code in PR #3085:
URL: 
https://github.com/apache/incubator-seatunnel/pull/3085#discussion_r996322486


##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -74,6 +92,7 @@ public KafkaSinkWriter(
             List<KafkaSinkState> kafkaStates) {
         this.context = context;
         this.pluginConfig = pluginConfig;
+        this.seaTunnelRowType = seaTunnelRowType;

Review Comment:
   ```suggestion
           this.partitionExtractor = createPartitionExtractor(pluginConfig, 
seaTunnelRowType);
   ```
   
   
   Add this method
   
   ```java
       private Function<SeaTunnelRow, String> createPartitionExtractor(Config 
pluginConfig,
                                                                       
SeaTunnelRowType seaTunnelRowType) {
           if (!pluginConfig.hasPath(PARTITION_KEY)){
               return row -> null;
           }
           String partitionKey = pluginConfig.getString(PARTITION_KEY);
           List<String> fieldNames = 
Arrays.asList(seaTunnelRowType.getFieldNames());
           if (!fieldNames.contains(partitionKey)) {
               return row -> partitionKey;
           }
           int partitionFieldIndex = seaTunnelRowType.indexOf(partitionKey);
           return row -> {
               Object partitionFieldValue = row.getField(partitionFieldIndex);
               if (partitionFieldValue != null) {
                   return partitionFieldValue.toString();
               }
               return null;
           };
       }
   ```



##########
seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/sink/KafkaSinkWriter.java:
##########
@@ -63,7 +66,22 @@ public class KafkaSinkWriter implements 
SinkWriter<SeaTunnelRow, KafkaCommitInfo
     // check config
     @Override
     public void write(SeaTunnelRow element) {
-        ProducerRecord<byte[], byte[]> producerRecord = 
seaTunnelRowSerializer.serializeRow(element);
+        ProducerRecord<byte[], byte[]> producerRecord = null;
+        //Determine the partition of the kafka send message based on the field 
name
+        if (pluginConfig.hasPath(PARTITION_KEY)){
+            String keyField = pluginConfig.getString(PARTITION_KEY);
+            List<String> fields = 
Arrays.asList(seaTunnelRowType.getFieldNames());
+            String key;
+            if (fields.contains(keyField)) {
+                key = element.getField(fields.indexOf(keyField)).toString();
+            } else {
+                key = keyField;
+            }

Review Comment:
   ```suggestion
               String key = partitionExtractor.apply(element);
   ```



-- 
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]

Reply via email to