This is an automated email from the ASF dual-hosted git repository.

fanjia pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git


The following commit(s) were added to refs/heads/dev by this push:
     new 142aca7b70 [Fix][Connecotr-kafka] Fix kafka IllegalArgumentException 
when offset is -1 (#9376)
142aca7b70 is described below

commit 142aca7b70e701ef6ac1e7a3cde96707b2995e9e
Author: WenDing-Y <[email protected]>
AuthorDate: Tue Jun 3 15:38:51 2025 +0800

    [Fix][Connecotr-kafka] Fix kafka IllegalArgumentException when offset is -1 
(#9376)
---
 .../serialize/DefaultSeaTunnelRowSerializer.java   |  4 +-
 .../kafka/source/KafkaSourceSplitEnumerator.java   |  4 ++
 .../seatunnel/e2e/connector/kafka/KafkaIT.java     | 57 +++++++++++++--
 ...source_timestamp_to_console_skip_partition.conf | 84 ++++++++++++++++++++++
 4 files changed, 140 insertions(+), 9 deletions(-)

diff --git 
a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
 
b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
index 70d08e5221..9ae2e554d8 100644
--- 
a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
+++ 
b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java
@@ -92,7 +92,7 @@ public class DefaultSeaTunnelRowSerializer implements 
SeaTunnelRowSerializer {
                 headersExtractor(rowType));
     }
 
-    public static DefaultSeaTunnelRowSerializer create(
+    public static DefaultSeaTunnelRowSerializer 
createWithPartitionAndTimestampFields(
             String topic,
             MessageFormat format,
             SeaTunnelRowType rowType,
@@ -100,7 +100,7 @@ public class DefaultSeaTunnelRowSerializer implements 
SeaTunnelRowSerializer {
             ReadonlyConfig pluginConfig) {
         return new DefaultSeaTunnelRowSerializer(
                 topicExtractor(topic, rowType, format),
-                partitionExtractor(null),
+                partitionNativeExtractor(rowType),
                 timestampExtractor(rowType),
                 keyExtractor(null, rowType, format, null, null),
                 valueExtractor(rowType, format, delimiter, pluginConfig),
diff --git 
a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java
 
b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java
index 686e0b71f3..e297cad40c 100644
--- 
a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java
+++ 
b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceSplitEnumerator.java
@@ -204,6 +204,10 @@ public class KafkaSourceSplitEnumerator
                     if (pendingSplit.containsKey(key)) {
                         pendingSplit.get(key).setStartOffset(value);
                     }
+                    if (!isStreamingMode && value < 0) {
+                        log.info("Skipping partition {} due to offset being 
-1", key);
+                        pendingSplit.remove(key);
+                    }
                 });
         if (!isStreamingMode && !topicPartitionEndOffsets.isEmpty()) {
             topicPartitionEndOffsets.forEach(
diff --git 
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java
 
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java
index 728c5ee8e0..bc3d4f2ed5 100644
--- 
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java
+++ 
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/KafkaIT.java
@@ -38,6 +38,7 @@ 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.utils.JsonUtils;
+import 
org.apache.seatunnel.connectors.seatunnel.kafka.config.KafkaBaseConstants;
 import org.apache.seatunnel.connectors.seatunnel.kafka.config.MessageFormat;
 import 
org.apache.seatunnel.connectors.seatunnel.kafka.serialize.DefaultSeaTunnelRowSerializer;
 import org.apache.seatunnel.e2e.common.TestResource;
@@ -160,9 +161,16 @@ public class KafkaIT extends TestSuiteBase implements 
TestResource {
                     new NewTopic("test_topic_source_timestamp", 1, (short) 1);
             
testTopicSourceWithTimestamp.configs(Collections.singletonMap("retention.ms", 
"-1"));
 
+            NewTopic testTopicSourceSkipPartition =
+                    new NewTopic("test_topic_source_skip_partition", 2, 
(short) 1);
+            
testTopicSourceSkipPartition.configs(Collections.singletonMap("retention.ms", 
"-1"));
+
             List<NewTopic> topics =
                     Arrays.asList(
-                            testTopicSource, testTopicNativeSource, 
testTopicSourceWithTimestamp);
+                            testTopicSource,
+                            testTopicNativeSource,
+                            testTopicSourceWithTimestamp,
+                            testTopicSourceSkipPartition);
             adminClient.createTopics(topics);
         }
 
@@ -177,15 +185,35 @@ public class KafkaIT extends TestSuiteBase implements 
TestResource {
         generateTestData(serializer::serializeRow, 0, 100);
 
         DefaultSeaTunnelRowSerializer rowSerializer =
-                DefaultSeaTunnelRowSerializer.create(
+                
DefaultSeaTunnelRowSerializer.createWithPartitionAndTimestampFields(
                         "test_topic_source_timestamp",
                         DEFAULT_FORMAT,
                         new SeaTunnelRowType(
-                                new String[] {"id", "timestamp"},
-                                new SeaTunnelDataType[] {BasicType.LONG_TYPE, 
BasicType.LONG_TYPE}),
+                                new String[] {"id", "timestamp", 
KafkaBaseConstants.PARTITION},
+                                new SeaTunnelDataType[] {
+                                    BasicType.LONG_TYPE, BasicType.LONG_TYPE, 
BasicType.INT_TYPE
+                                }),
                         "",
                         null);
-        generateWithTimestampTestData(rowSerializer::serializeRow, 0, 100, 
1738395840000L);
+
+        DefaultSeaTunnelRowSerializer topicSourceSkipPartition =
+                
DefaultSeaTunnelRowSerializer.createWithPartitionAndTimestampFields(
+                        "test_topic_source_skip_partition",
+                        DEFAULT_FORMAT,
+                        new SeaTunnelRowType(
+                                new String[] {"id", "timestamp", 
KafkaBaseConstants.PARTITION},
+                                new SeaTunnelDataType[] {
+                                    BasicType.LONG_TYPE, BasicType.LONG_TYPE, 
BasicType.INT_TYPE
+                                }),
+                        "",
+                        null);
+
+        generateWithTimestampTestData(rowSerializer::serializeRow, 0, 100, 
1738395840000L, 0);
+
+        generateWithTimestampTestData(
+                topicSourceSkipPartition::serializeRow, 0, 100, 
1738395840000L, 0);
+        generateWithTimestampTestData(
+                topicSourceSkipPartition::serializeRow, 100, 200, 
1738396200000L, 1);
 
         String topicName = "test_topic_native_source";
         generateNativeTestData("test_topic_native_source", 0, 100);
@@ -451,6 +479,14 @@ public class KafkaIT extends TestSuiteBase implements 
TestResource {
         testKafkaWithEndTimestampToConsole(container);
     }
 
+    @TestTemplate
+    public void testSourceKafkaSkipPartition(TestContainer container)
+            throws IOException, InterruptedException {
+        Container.ExecResult execResult =
+                
container.executeJob("/kafka/kafkasource_timestamp_to_console_skip_partition.conf");
+        Assertions.assertEquals(0, execResult.getExitCode(), 
execResult.getStderr());
+    }
+
     @TestTemplate
     public void testSourceKafkaStartConfig(TestContainer container)
             throws IOException, InterruptedException {
@@ -1237,11 +1273,18 @@ public class KafkaIT extends TestSuiteBase implements 
TestResource {
     }
 
     private void generateWithTimestampTestData(
-            ProducerRecordConverter converter, int start, int end, long 
startTimestamp) {
+            ProducerRecordConverter converter,
+            int start,
+            int end,
+            long startTimestamp,
+            int partition) {
         try {
             for (int i = start; i < end; i++) {
                 SeaTunnelRow row =
-                        new SeaTunnelRow(new Object[] {Long.valueOf(i), 
startTimestamp + i * 1000});
+                        new SeaTunnelRow(
+                                new Object[] {
+                                    Long.valueOf(i), startTimestamp + i * 
1000, partition
+                                });
                 ProducerRecord<byte[], byte[]> producerRecord = 
converter.convert(row);
                 producer.send(producerRecord).get();
             }
diff --git 
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_timestamp_to_console_skip_partition.conf
 
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_timestamp_to_console_skip_partition.conf
new file mode 100644
index 0000000000..1ed3533df5
--- /dev/null
+++ 
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafka/kafkasource_timestamp_to_console_skip_partition.conf
@@ -0,0 +1,84 @@
+
+#
+# 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 {
+  parallelism = 1
+  job.mode = "BATCH"
+}
+
+source {
+  Kafka {
+    bootstrap.servers = "kafkaCluster:9092"
+    topic = "test_topic_source_skip_partition"
+    plugin_output = "kafka_table"
+    # The default format is json, which is optional
+    format = json
+    start_mode = timestamp
+    schema = {
+      fields {
+        id = bigint
+      }
+    }
+    start_mode.timestamp = 1738396301000
+  }
+
+  # 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 {
+}
+
+sink {
+  Assert {
+    plugin_input = "kafka_table"
+    rules =
+      {
+        field_rules = [
+          {
+            field_name = id
+            field_type = bigint
+            field_value = [
+
+              {
+                rule_type = MIN
+                rule_value = 101
+              },
+              {
+                rule_type = MAX
+                rule_value = 199
+              }
+            ]
+          }
+        ]
+        row_rules = [
+                        {
+                          rule_type = MIN_ROW
+                          rule_value = 99
+                        },
+                        {
+                          rule_type = MAX_ROW
+                          rule_value = 99
+                        }
+                      ]
+      }
+  }
+}

Reply via email to