This is an automated email from the ASF dual-hosted git repository.
corgy 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 9f76fa5398 [Fix][Connector-V2][SqlServer-CDC] Fix earliest
startup.mode offset and align tests (#10207)
9f76fa5398 is described below
commit 9f76fa539890a49d23eb5a964747ecb28bd282db
Author: yzeng1618 <[email protected]>
AuthorDate: Mon Dec 22 14:17:47 2025 +0800
[Fix][Connector-V2][SqlServer-CDC] Fix earliest startup.mode offset and
align tests (#10207)
Co-authored-by: zengyi <[email protected]>
---
.../cdc/sqlserver/source/offset/LsnOffset.java | 2 +-
.../cdc/sqlserver/source/offset/LsnOffsetTest.java | 38 ++++++++++++++
.../connector/cdc/sqlserver/SqlServerCDCIT.java | 41 +++++++++++++++
.../sqlservercdc_earliest_to_sqlserver.conf | 59 ++++++++++++++++++++++
4 files changed, 139 insertions(+), 1 deletion(-)
diff --git
a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffset.java
b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffset.java
index d4b517e48a..7fa7a59f88 100644
---
a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffset.java
+++
b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffset.java
@@ -29,7 +29,7 @@ public class LsnOffset extends Offset {
private static final long serialVersionUID = 1L;
- public static final LsnOffset INITIAL_OFFSET = valueOf(Lsn.valueOf(new
byte[] {0}).toString());
+ public static final LsnOffset INITIAL_OFFSET = new LsnOffset(null, null,
null);
public static final LsnOffset NO_STOPPING_OFFSET =
valueOf(Lsn.valueOf(new byte[] {Byte.MAX_VALUE}).toString());
diff --git
a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffsetTest.java
b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffsetTest.java
new file mode 100644
index 0000000000..7c62cb46c5
--- /dev/null
+++
b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/offset/LsnOffsetTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+package org.apache.seatunnel.connectors.seatunnel.cdc.sqlserver.source.offset;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import io.debezium.connector.sqlserver.Lsn;
+
+class LsnOffsetTest {
+
+ @Test
+ void testInitialOffsetRepresentsNoLsn() {
+ LsnOffset initial = LsnOffset.INITIAL_OFFSET;
+
+ // no LSN keys should be present in the offset map
+ Assertions.assertTrue(initial.getOffset().isEmpty());
+
+ // commit LSN resolved from the empty map should be Debezium's NULL LSN
+ Lsn commitLsn = initial.getCommitLsn();
+ Assertions.assertFalse(commitLsn.isAvailable());
+ }
+}
diff --git
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/sqlserver/SqlServerCDCIT.java
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/sqlserver/SqlServerCDCIT.java
index 11083fcf89..ae9c01ce4e 100644
---
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/sqlserver/SqlServerCDCIT.java
+++
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/sqlserver/SqlServerCDCIT.java
@@ -316,6 +316,47 @@ public class SqlServerCDCIT extends TestSuiteBase
implements TestResource {
});
}
+ @TestTemplate
+ @DisabledOnContainer(
+ value = {},
+ type = {EngineType.SPARK, EngineType.FLINK},
+ disabledReason =
+ "This case checks SqlServer CDC earliest startup mode only
on Zeta engine.")
+ public void testEarliestStartupMode(TestContainer container) throws
InterruptedException {
+ initializeSqlServerTable("column_type_test");
+
+ Long jobId = JobIdGenerator.newJobId();
+ CompletableFuture.runAsync(
+ () -> {
+ try {
+ container.executeJob(
+ "/sqlservercdc_earliest_to_sqlserver.conf",
String.valueOf(jobId));
+ } catch (Exception e) {
+ log.error("Execute earliest job exception: {}",
e.getMessage());
+ throw new RuntimeException(e);
+ }
+ });
+
+ // give the job some time to start
+ TimeUnit.SECONDS.sleep(10);
+
+ // verify job stays running (i.e. no fatal exception like
ArrayIndexOutOfBounds from
+ // Debezium)
+ await().atMost(2, TimeUnit.MINUTES)
+ .untilAsserted(
+ () -> {
+ String jobStatus =
container.getJobStatus(String.valueOf(jobId));
+ Assertions.assertEquals("RUNNING", jobStatus);
+ });
+
+ try {
+ Container.ExecResult cancelJobResult =
container.cancelJob(String.valueOf(jobId));
+ Assertions.assertEquals(0, cancelJobResult.getExitCode(),
cancelJobResult.getStderr());
+ } catch (IOException | InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
@TestTemplate
@DisabledOnContainer(
value = {},
diff --git
a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_earliest_to_sqlserver.conf
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_earliest_to_sqlserver.conf
new file mode 100644
index 0000000000..6dfef86e9f
--- /dev/null
+++
b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_earliest_to_sqlserver.conf
@@ -0,0 +1,59 @@
+#
+# 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 {
+ # You can set engine configuration here
+ parallelism = 1
+ job.mode = "STREAMING"
+ checkpoint.interval = 5000
+}
+
+source {
+ # This is an example source plugin **only for test and demonstrate the
feature source plugin**
+ SqlServer-CDC {
+ plugin_output = "customers"
+ username = "sa"
+ password = "Password!"
+ database-names = ["column_type_test"]
+ table-names = ["column_type_test.dbo.full_types"]
+ url = "jdbc:sqlserver://sqlserver-host:1433;databaseName=column_type_test"
+ # start from the earliest available CDC LSN
+ startup.mode = "earliest"
+ }
+}
+
+transform {
+}
+
+sink {
+ Jdbc {
+ plugin_input = "customers"
+ driver = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+ url = "jdbc:sqlserver://sqlserver-host:1433;encrypt=false"
+ user = "sa"
+ password = "Password!"
+ generate_sink_sql = true
+ database = "column_type_test"
+ table = "dbo.full_types_sink"
+ batch_size = 1
+ primary_keys = ["id"]
+ }
+}
+