From d0fd0b12b8586d5469f8a4f0d0fce10f29ae762b Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 18:48:49 +0800 Subject: [PATCH 01/22] [Improve](streaming-job) support specifying offset for StreamingInsertJob create and alter Co-Authored-By: Claude Opus 4.6 (1M context) --- .../streaming/DataSourceConfigValidator.java | 54 +++- .../insert/streaming/StreamingInsertJob.java | 13 +- .../doris/job/offset/jdbc/JdbcOffset.java | 7 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 24 +- .../trees/plans/commands/AlterJobCommand.java | 3 +- .../plans/commands/CreateJobCommand.java | 3 +- .../reader/JdbcIncrementalSourceReader.java | 4 +- .../reader/postgres/PostgresSourceReader.java | 11 +- ..._streaming_mysql_job_special_offset.groovy | 229 ++++++++++++++ ...mysql_job_special_offset_restart_fe.groovy | 140 +++++++++ ...reaming_postgres_job_special_offset.groovy | 296 ++++++++++++++++++ 11 files changed, 764 insertions(+), 20 deletions(-) create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java index b75e202b1a842e..3700b80e6a6e30 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -18,14 +18,19 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; import java.util.Map; import java.util.Set; public class DataSourceConfigValidator { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Set ALLOW_SOURCE_KEYS = Sets.newHashSet( DataSourceConfigKeys.JDBC_URL, DataSourceConfigKeys.USER, @@ -51,7 +56,7 @@ public class DataSourceConfigValidator { private static final String TABLE_LEVEL_PREFIX = DataSourceConfigKeys.TABLE + "."; - public static void validateSource(Map input) throws IllegalArgumentException { + public static void validateSource(Map input, String dataSourceType) throws IllegalArgumentException { for (Map.Entry entry : input.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); @@ -79,7 +84,7 @@ public static void validateSource(Map input) throws IllegalArgum throw new IllegalArgumentException("Unexpected key: '" + key + "'"); } - if (!isValidValue(key, value)) { + if (!isValidValue(key, value, dataSourceType)) { throw new IllegalArgumentException("Invalid value for key '" + key + "': " + value); } } @@ -103,18 +108,51 @@ public static void validateTarget(Map input) throws IllegalArgum } } - private static boolean isValidValue(String key, String value) { + private static boolean isValidValue(String key, String value, String dataSourceType) { if (value == null || value.isEmpty()) { return false; } - if (key.equals(DataSourceConfigKeys.OFFSET) - && !(value.equals(DataSourceConfigKeys.OFFSET_INITIAL) - || value.equals(DataSourceConfigKeys.OFFSET_LATEST) - || value.equals(DataSourceConfigKeys.OFFSET_SNAPSHOT))) { - return false; + if (key.equals(DataSourceConfigKeys.OFFSET)) { + return isValidOffset(value, dataSourceType); } return true; } + /** + * Check if the offset value is valid for the given data source type. + * Supported: initial, snapshot, latest, JSON binlog/lsn position. + * earliest is only supported for MySQL. + */ + public static boolean isValidOffset(String offset, String dataSourceType) { + if (offset == null || offset.isEmpty()) { + return false; + } + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset)) { + return true; + } + // earliest only for MySQL + if (DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset)) { + return DataSourceType.MYSQL.name().equalsIgnoreCase(dataSourceType); + } + if (isJsonOffset(offset)) { + return true; + } + return false; + } + + public static boolean isJsonOffset(String offset) { + if (offset == null || offset.trim().isEmpty()) { + return false; + } + try { + JsonNode node = OBJECT_MAPPER.readTree(offset); + return node.isObject(); + } catch (Exception e) { + return false; + } + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index d7a325e22f541e..dbf975c3292df7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -306,9 +306,8 @@ private void initInsertJob() { this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); this.offsetProvider.ensureInitialized(getJobId(), originTvfProps); this.offsetProvider.initOnCreate(); - // validate offset props, only for s3 cause s3 tvf no offset prop - if (jobProperties.getOffsetProperty() != null - && S3TableValuedFunction.NAME.equalsIgnoreCase(tvfType)) { + // validate offset props + if (jobProperties.getOffsetProperty() != null) { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } @@ -780,13 +779,17 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { */ private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); - if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty()) - && S3TableValuedFunction.NAME.equalsIgnoreCase(this.tvfType)) { + if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); if (Config.isCloudMode()) { resetCloudProgress(offset); } + // For FROM...TO path, also update sourceProperties so the CDC client + // uses the new offset when building the next task. + if (this.sourceProperties != null) { + this.sourceProperties.put(DataSourceConfigKeys.OFFSET, inputStreamProps.getOffsetProperty()); + } } this.properties.putAll(inputProperties); this.jobProperties = new StreamingJobProperties(this.properties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index dea2c244d6d78c..faf39dbe30899d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -48,7 +48,10 @@ public class JdbcOffset implements Offset { @Override public String toSerializedJson() { - return null; + if (splits == null || splits.isEmpty()) { + return null; + } + return new Gson().toJson(splits); } @Override @@ -58,7 +61,7 @@ public boolean isEmpty() { @Override public boolean isValidOffset() { - return false; + return splits != null && !splits.isEmpty(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index f6bbdcc5b56386..75a8c880684e74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -29,6 +29,7 @@ import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; @@ -360,7 +361,28 @@ public Offset deserializeOffset(String offset) { @Override public Offset deserializeOffsetProperty(String offset) { - // no need cause cdc_stream has offset property + if (offset == null || offset.trim().isEmpty()) { + return null; + } + // Named modes: stored in sourceProperties.offset, CDC client reads it directly. + // Return a placeholder JdbcOffset so validateOffset() passes. + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset)) { + return new JdbcOffset(Collections.singletonList(new BinlogSplit())); + } + // JSON format: {"file":"binlog.000003","pos":154} or {"lsn":"123456"} + if (DataSourceConfigValidator.isJsonOffset(offset)) { + try { + Map offsetMap = objectMapper.readValue(offset, + new TypeReference>() {}); + return new JdbcOffset(Collections.singletonList(new BinlogSplit(offsetMap))); + } catch (Exception e) { + log.warn("Failed to parse JSON offset: {}", offset, e); + return null; + } + } return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index fc63710f93ddf1..c0455ab6c1c504 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -166,7 +166,8 @@ private void validate() throws Exception { boolean sourcePropModified = isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); if (sourcePropModified) { - DataSourceConfigValidator.validateSource(this.getSourceProperties()); + DataSourceConfigValidator.validateSource(this.getSourceProperties(), + streamingJob.getSourceType()); checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index 1640143d277f01..ff278293e8216d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -74,7 +74,8 @@ private void validate() throws JobException { } if (StringUtils.isNotEmpty(createJobInfo.getSourceType())) { - DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties()); + DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties(), + createJobInfo.getSourceType()); DataSourceConfigValidator.validateTarget(createJobInfo.getTargetProperties()); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 3d9167d6124f1d..3c0e6a8d001f8c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -720,8 +720,10 @@ private Offset getStartOffsetFromConfig(JdbcSourceConfig sourceConfig) { case EARLIEST_OFFSET: startingOffset = createInitialOffset(); break; - case TIMESTAMP: case SPECIFIC_OFFSETS: + startingOffset = createOffset(startupOptions.getOffset()); + break; + case TIMESTAMP: case COMMITTED_OFFSETS: default: throw new IllegalStateException( diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index b6d28510613c94..e4fcd44b35015c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -206,7 +206,16 @@ private PostgresSourceConfig generatePostgresConfig( } else if (DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.latest()); } else if (ConfigUtil.isJson(startupMode)) { - throw new RuntimeException("Unsupported json offset " + startupMode); + Map offsetMap = ConfigUtil.toStringMap(startupMode); + if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { + throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); + } + // Ensure ts_usec is present (required by PostgresOffset) + if (!offsetMap.containsKey(SourceInfo.TIMESTAMP_USEC_KEY)) { + offsetMap.put(SourceInfo.TIMESTAMP_USEC_KEY, + String.valueOf(Conversions.toEpochMicros(Instant.MIN))); + } + configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { // start from timestamp Long ts = Long.parseLong(startupMode); diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy new file mode 100644 index 00000000000000..da9331e524feed --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -0,0 +1,229 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_mysql_job_special_offset" + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_mysql_tbl" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `id` int NOT NULL, + `name` varchar(100) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // ===== Test 1: offset = earliest, verify data synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "earliest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" + assert rows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 2: offset = latest, then insert new data, verify synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // wait for job to be running + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // insert new data after job started with latest offset + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 1 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3: ALTER JOB to change offset ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // pause, then alter offset to initial + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to initial, snapshot data should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 3 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 4: invalid offset format ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "not_valid_offset" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 5: JSON binlog offset, verify data synced ===== + // Get current binlog position, insert data after it, then create job from that position + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Current binlog position: file=${binlogFile}, pos=${binlogPos}") + // insert data after this binlog position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (10, 'specific1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (11, 'specific2')""" + } + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("Using JSON offset: ${offsetJson}") + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = '${offsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def specificRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (10, 11) order by id""" + log.info("specificRows: " + specificRows) + assert specificRows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // cleanup MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + } + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy new file mode 100644 index 00000000000000..9b89d43334fa0f --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy @@ -0,0 +1,140 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_special_offset_restart_fe", "docker,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_mysql_job_special_offset_restart_fe" + def options = new ClusterOptions() + options.setFeNum(1) + // run in cloud and not cloud + options.cloudMode = null + + docker(options) { + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_restart_tbl" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare MySQL source table and get binlog position + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `id` int NOT NULL, + `name` varchar(100) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + // get current binlog position + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Binlog position: file=${binlogFile}, pos=${binlogPos}") + // insert data after this position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // create job with JSON binlog offset + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("Creating job with offset: ${offsetJson}") + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = '${offsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // wait for data synced + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex) { + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoBefore = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoBefore: " + jobInfoBefore) + assert jobInfoBefore.get(0).get(1) == "RUNNING" + + // Restart FE + cluster.restartFrontends() + sleep(60000) + context.reconnectFe() + + // check job is consistent after restart + def jobInfoAfter = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoAfter: " + jobInfoAfter) + assert jobInfoAfter.get(0).get(1) == "RUNNING" + assert jobInfoAfter.get(0).get(2) == jobInfoBefore.get(0).get(2) + + // insert more data and verify job still works after restart + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 3 + }) + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + } + } + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy new file mode 100644 index 00000000000000..6244b62b98b1d6 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -0,0 +1,296 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_docker,external_docker_pg,nondatalake") { + def jobName = "test_streaming_pg_job_special_offset" + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_pg_tbl" + def pgDB = "postgres" + def pgSchema = "cdc_test" + def pgUser = "postgres" + def pgPassword = "123456" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String pg_port = context.config.otherConfigs.get("pg_14_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/postgresql-42.5.0.jar" + + // prepare PG source table + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" + sql """CREATE TABLE ${pgDB}.${pgSchema}.${table1} ( + "id" int, + "name" varchar(100), + PRIMARY KEY ("id") + )""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // ===== Test 1: offset = initial, verify data synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" + assert rows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 2: offset = latest, then insert new data, verify synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // insert new data after job started with latest offset + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 1 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3: ALTER JOB to change offset ===== + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DELETE FROM ${pgDB}.${pgSchema}.${table1} WHERE id = 3""" + } + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // pause, then alter offset to initial + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to initial, snapshot data should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 4: earliest should fail for PG ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "earliest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 5: invalid offset format ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "not_valid" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 6: JSON LSN offset via ALTER, verify data synced ===== + // Step 1: Create job with initial to establish replication slot and sync snapshot + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + + // Step 2: Get current WAL LSN, then insert new data + def currentLsn = "" + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + def lsnResult = sql """SELECT pg_current_wal_lsn()::text""" + def lsnStr = lsnResult[0][0].toString() + // Convert PG LSN format (e.g. "0/1A3B4C0") to numeric + def parts = lsnStr.split("/") + def high = Long.parseLong(parts[0], 16) + def low = Long.parseLong(parts[1], 16) + currentLsn = String.valueOf((high << 32) + low) + log.info("Current WAL LSN: ${lsnStr} -> numeric: ${currentLsn}") + // insert new data after this LSN + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (20, 'lsn_test1')""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (21, 'lsn_test2')""" + } + + // Step 3: PAUSE -> ALTER with JSON LSN -> RESUME + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + def lsnOffsetJson = """{"lsn":"${currentLsn}"}""" + log.info("Using JSON LSN offset: ${lsnOffsetJson}") + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = '${lsnOffsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + + // Step 4: Verify new data synced + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" + return result[0][0] >= 2 + }) + def lsnRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (20, 21) order by id""" + log.info("lsnRows: " + lsnRows) + assert lsnRows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // cleanup PG source table + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" + } + } +} From 44f21e4d0cb9cfb6be6b6d9788a94bb78a1727ac Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 19:14:42 +0800 Subject: [PATCH 02/22] [Improve](streaming-job) ALTER CDC offset only supports JSON, add TVF alter offset test - CDC ALTER only allows JSON specific offset (reject named modes like initial/latest) - ALTER offset uses PROPERTIES('offset'='{"file":"xxx","pos":"yyy"}') syntax - Update regression cases to use PROPERTIES for ALTER - Add cdc_stream TVF ALTER offset regression test Co-Authored-By: Claude Opus 4.6 (1M context) --- .../insert/streaming/StreamingInsertJob.java | 14 +- ..._streaming_mysql_job_special_offset.groovy | 51 +++++- ...reaming_postgres_job_special_offset.groovy | 51 +----- ...g_job_cdc_stream_mysql_alter_offset.groovy | 155 ++++++++++++++++++ 4 files changed, 217 insertions(+), 54 deletions(-) create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index dbf975c3292df7..e8df49c1befec8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -780,16 +780,20 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { + // For CDC jobs, ALTER only supports JSON specific offset (e.g. binlog position or LSN), + // named modes like initial/latest/snapshot are only valid at CREATE time. + if (offsetProvider instanceof JdbcSourceOffsetProvider + && !DataSourceConfigValidator.isJsonOffset(inputStreamProps.getOffsetProperty())) { + throw new AnalysisException( + "ALTER JOB for CDC only supports JSON specific offset, " + + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " + + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); + } Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); if (Config.isCloudMode()) { resetCloudProgress(offset); } - // For FROM...TO path, also update sourceProperties so the CDC client - // uses the new offset when building the next task. - if (this.sourceProperties != null) { - this.sourceProperties.put(DataSourceConfigKeys.OFFSET, inputStreamProps.getOffsetProperty()); - } } this.properties.putAll(inputProperties); this.jobProperties = new StreamingJobProperties(this.properties); diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index da9331e524feed..c7757e3094f4cc 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -108,7 +108,19 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER JOB to change offset ===== + // ===== Test 3: ALTER JOB with JSON binlog offset via PROPERTIES ===== + // Get current binlog position, then create job with initial + def alterBinlogFile = "" + def alterBinlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + alterBinlogFile = masterStatus[0][0] + alterBinlogPos = masterStatus[0][1].toString() + log.info("ALTER test binlog position: file=${alterBinlogFile}, pos=${alterBinlogPos}") + // insert data after this position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (20, 'alter_test1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (21, 'alter_test2')""" + } sql """CREATE JOB ${jobName} ON STREAMING FROM MYSQL ( @@ -129,13 +141,29 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) - // pause, then alter offset to initial + // pause, then alter offset to specific binlog position via PROPERTIES sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus[0][0] == "PAUSED" }) + def alterOffsetJson = """{"file":"${alterBinlogFile}","pos":"${alterBinlogPos}"}""" + log.info("ALTER offset: ${alterOffsetJson}") sql """ALTER JOB ${jobName} + PROPERTIES('offset' = '${alterOffsetJson}') + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to specific binlog position, data inserted after that position should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" + return result[0][0] >= 2 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3b: ALTER with named mode should fail for CDC ===== + sql """CREATE JOB ${jobName} + ON STREAMING FROM MYSQL ( "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", "driver_url" = "${driver_url}", @@ -150,12 +178,21 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc "table.create.properties.replication_num" = "1" ) """ - sql "RESUME JOB where jobname = '${jobName}'" - // after alter to initial, snapshot data should sync - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 3 + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + test { + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = 'initial') + """ + exception "ALTER JOB for CDC only supports JSON specific offset" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 6244b62b98b1d6..2dc84e6648c984 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -111,10 +111,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER JOB to change offset ===== - connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { - sql """DELETE FROM ${pgDB}.${pgSchema}.${table1} WHERE id = 3""" - } + // ===== Test 3: ALTER with named mode should fail for CDC ===== sql """CREATE JOB ${jobName} ON STREAMING FROM POSTGRES ( @@ -126,7 +123,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc "database" = "${pgDB}", "schema" = "${pgSchema}", "include_tables" = "${table1}", - "offset" = "latest" + "offset" = "initial" ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" @@ -136,34 +133,17 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) - // pause, then alter offset to initial sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus[0][0] == "PAUSED" }) - sql """ALTER JOB ${jobName} - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - sql "RESUME JOB where jobname = '${jobName}'" - // after alter to initial, snapshot data should sync - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) + test { + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = 'initial') + """ + exception "ALTER JOB for CDC only supports JSON specific offset" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" @@ -260,20 +240,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc def lsnOffsetJson = """{"lsn":"${currentLsn}"}""" log.info("Using JSON LSN offset: ${lsnOffsetJson}") sql """ALTER JOB ${jobName} - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = '${lsnOffsetJson}' - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) + PROPERTIES('offset' = '${lsnOffsetJson}') """ sql "RESUME JOB where jobname = '${jobName}'" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy new file mode 100644 index 00000000000000..5158da072f8636 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy @@ -0,0 +1,155 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +/** + * Test ALTER JOB with JSON binlog offset for cdc_stream TVF path. + * + * Scenario: + * 1. Create job with initial offset, wait for snapshot sync. + * 2. Get current binlog position, insert new data. + * 3. PAUSE -> ALTER with JSON binlog offset via PROPERTIES -> RESUME. + * 4. Verify new data synced. + */ +suite("test_streaming_job_cdc_stream_mysql_alter_offset", "p0,external,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_job_cdc_stream_mysql_alter_offset_name" + def currentDb = (sql "select database()")[0][0] + def dorisTable = "cdc_stream_alter_offset_tbl" + def mysqlDb = "test_cdc_db" + def mysqlTable = "cdc_stream_alter_offset_src" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${dorisTable} force""" + + sql """ + CREATE TABLE IF NOT EXISTS ${currentDb}.${dorisTable} ( + `id` int NULL, + `name` varchar(200) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS AUTO + PROPERTIES ("replication_allocation" = "tag.location.default: 1") + """ + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare source table with snapshot data + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${mysqlTable}""" + sql """CREATE TABLE ${mysqlDb}.${mysqlTable} ( + `id` int NOT NULL, + `name` varchar(200) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (1, 'alice'), (2, 'bob')""" + } + + // Step 1: Create job with initial offset via cdc_stream TVF + sql """ + CREATE JOB ${jobName} + ON STREAMING DO INSERT INTO ${currentDb}.${dorisTable} (id, name) + SELECT id, name FROM cdc_stream( + "type" = "mysql", + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "table" = "${mysqlTable}", + "offset" = "initial" + ) + """ + + // wait for snapshot sync + try { + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ + def cnt = sql """select SucceedTaskCount from jobs("type"="insert") where Name='${jobName}' and ExecuteType='STREAMING'""" + log.info("SucceedTaskCount: " + cnt) + cnt.size() == 1 && (cnt.get(0).get(0) as int) >= 1 + }) + } catch (Exception ex) { + log.info("job: " + (sql """select * from jobs("type"="insert") where Name='${jobName}'""")) + log.info("tasks: " + (sql """select * from tasks("type"="insert") where JobName='${jobName}'""")) + throw ex + } + + // verify snapshot data + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable}""" + return result[0][0] >= 2 + }) + + // Step 2: Get current binlog position and insert new data + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Binlog position for ALTER: file=${binlogFile}, pos=${binlogPos}") + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (10, 'alter_tvf_1')""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (11, 'alter_tvf_2')""" + } + + // Step 3: PAUSE -> ALTER with JSON binlog offset -> RESUME + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("ALTER TVF job offset: ${offsetJson}") + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = '${offsetJson}') + """ + + // verify currentOffset changed in show jobs + def jobInfo = sql """select currentOffset from jobs("type"="insert") where Name='${jobName}'""" + log.info("currentOffset after ALTER: " + jobInfo[0][0]) + + sql "RESUME JOB where jobname = '${jobName}'" + + // Step 4: Verify new data synced + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11)""" + return result[0][0] >= 2 + }) + def alterRows = sql """SELECT * FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11) order by id""" + log.info("alterRows: " + alterRows) + assert alterRows.size() == 2 + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${dorisTable} force""" + + // cleanup MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${mysqlTable}""" + } + } +} From 14306bbe5446760991543c9e998c0997220caf52 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 19:15:36 +0800 Subject: [PATCH 03/22] [fix](streaming-job) fix checkstyle: line length and unused import Co-Authored-By: Claude Opus 4.6 (1M context) --- .../extensions/insert/streaming/DataSourceConfigValidator.java | 3 ++- .../job/extensions/insert/streaming/StreamingInsertJob.java | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java index 3700b80e6a6e30..5ef98668cbab12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -56,7 +56,8 @@ public class DataSourceConfigValidator { private static final String TABLE_LEVEL_PREFIX = DataSourceConfigKeys.TABLE + "."; - public static void validateSource(Map input, String dataSourceType) throws IllegalArgumentException { + public static void validateSource(Map input, + String dataSourceType) throws IllegalArgumentException { for (Map.Entry entry : input.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index e8df49c1befec8..3665cdc98217af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -73,7 +73,6 @@ import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; -import org.apache.doris.tablefunction.S3TableValuedFunction; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; From 7ccb97bfb17ea9c475a398e041461fae7d958bd1 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 10:14:23 +0800 Subject: [PATCH 04/22] [fix](streaming-job) fix AlterJobCommand compile error: use getDataSourceType().name() Co-Authored-By: Claude Opus 4.6 (1M context) --- .../doris/nereids/trees/plans/commands/AlterJobCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index c0455ab6c1c504..029bf932e5cc82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -167,7 +167,7 @@ private void validate() throws Exception { isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); if (sourcePropModified) { DataSourceConfigValidator.validateSource(this.getSourceProperties(), - streamingJob.getSourceType()); + streamingJob.getDataSourceType().name()); checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); } From 5360d8faae5d36cb96a5c68081c2947de7228759 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 11:27:32 +0800 Subject: [PATCH 05/22] [Improve](streaming-job) address review: validate ALTER offset in provider, fix serialization - Add validateAlterOffset() to SourceOffsetProvider interface, CDC impl rejects non-JSON offset, S3 impl allows any format (default no-op) - StreamingInsertJob delegates ALTER offset validation to provider - AlterJobCommand.validateProps calls validateAlterOffset before validateOffset - Restore S3-only guard in initInsertJob for PROPERTIES offset - Fix JdbcOffset.toSerializedJson() to produce flat format compatible with deserializeOffset(): [{"split_id":"binlog-split","file":"xxx","pos":"yyy"}] - Remove unnecessary ts_usec padding in PostgresSourceReader JSON offset - Remove instanceof check in modifyPropertiesInternal (validation moved to AlterJobCommand) Co-Authored-By: Claude Opus 4.6 (1M context) --- .../insert/streaming/StreamingInsertJob.java | 26 +++++++++++-------- .../job/offset/SourceOffsetProvider.java | 7 +++++ .../doris/job/offset/jdbc/JdbcOffset.java | 13 +++++++++- .../offset/jdbc/JdbcSourceOffsetProvider.java | 11 ++++++++ .../trees/plans/commands/AlterJobCommand.java | 5 ++-- .../reader/postgres/PostgresSourceReader.java | 5 ---- 6 files changed, 47 insertions(+), 20 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 3665cdc98217af..f6460db6c16dd1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -73,6 +73,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; +import org.apache.doris.tablefunction.S3TableValuedFunction; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; @@ -305,8 +306,9 @@ private void initInsertJob() { this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); this.offsetProvider.ensureInitialized(getJobId(), originTvfProps); this.offsetProvider.initOnCreate(); - // validate offset props - if (jobProperties.getOffsetProperty() != null) { + // validate offset props, only for s3 cause s3 tvf no offset prop + if (jobProperties.getOffsetProperty() != null + && S3TableValuedFunction.NAME.equalsIgnoreCase(tvfType)) { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } @@ -357,6 +359,17 @@ public void initLogicalPlan(boolean regen) { } } + /** + * Validate the offset format for ALTER JOB, delegating to the provider. + */ + public void validateAlterOffset(String offset) throws AnalysisException { + try { + offsetProvider.validateAlterOffset(offset); + } catch (Exception ex) { + throw new AnalysisException(ex.getMessage()); + } + } + /** * Check whether Offset can be serialized into the corresponding data source * */ @@ -779,15 +792,6 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { - // For CDC jobs, ALTER only supports JSON specific offset (e.g. binlog position or LSN), - // named modes like initial/latest/snapshot are only valid at CREATE time. - if (offsetProvider instanceof JdbcSourceOffsetProvider - && !DataSourceConfigValidator.isJsonOffset(inputStreamProps.getOffsetProperty())) { - throw new AnalysisException( - "ALTER JOB for CDC only supports JSON specific offset, " - + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " - + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); - } Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); if (Config.isCloudMode()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index 33a24378704fb6..4c44c093f79748 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -114,6 +114,13 @@ default void initOnCreate() throws JobException {} */ Offset deserializeOffsetProperty(String offset); + /** + * Validate the offset format for ALTER JOB. + * Each provider defines its own rules (e.g. CDC only allows JSON specific offset). + */ + default void validateAlterOffset(String offset) throws Exception { + } + /** * Replaying OffsetProvider is currently only required by JDBC. * diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index faf39dbe30899d..d6a29900a6c524 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -32,6 +32,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,7 +52,17 @@ public String toSerializedJson() { if (splits == null || splits.isEmpty()) { return null; } - return new Gson().toJson(splits); + // Serialize to the same flat format that deserializeOffset() expects: + // [{"split_id":"binlog-split","file":"xxx","pos":"yyy"}] + Preconditions.checkState(splits.size() == 1 && splits.get(0) instanceof BinlogSplit, + "toSerializedJson only supports single BinlogSplit"); + BinlogSplit binlog = (BinlogSplit) splits.get(0); + Map map = new HashMap<>(); + map.put(JdbcSourceOffsetProvider.SPLIT_ID, binlog.getSplitId()); + if (binlog.getStartingOffset() != null) { + map.putAll(binlog.getStartingOffset()); + } + return new Gson().toJson(Collections.singletonList(map)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 75a8c880684e74..5b876a4baffa25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -18,6 +18,7 @@ package org.apache.doris.job.offset.jdbc; import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.job.cdc.DataSourceConfigKeys; @@ -386,6 +387,16 @@ public Offset deserializeOffsetProperty(String offset) { return null; } + @Override + public void validateAlterOffset(String offset) throws Exception { + if (!DataSourceConfigValidator.isJsonOffset(offset)) { + throw new AnalysisException( + "ALTER JOB for CDC only supports JSON specific offset, " + + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " + + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); + } + } + /** * Replay snapshot splits if needed */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 029bf932e5cc82..7974c7751e03ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -218,9 +218,8 @@ private void checkUnmodifiableSourceProperties(Map originSourceP private void validateProps(StreamingInsertJob streamingJob) throws AnalysisException { StreamingJobProperties jobProperties = new StreamingJobProperties(properties); jobProperties.validate(); - // from to job no need valiate offset in job properties - if (streamingJob.getDataSourceType() == null - && jobProperties.getOffsetProperty() != null) { + if (jobProperties.getOffsetProperty() != null) { + streamingJob.validateAlterOffset(jobProperties.getOffsetProperty()); streamingJob.validateOffset(jobProperties.getOffsetProperty()); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index e4fcd44b35015c..f1a0c572f24dcf 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -210,11 +210,6 @@ private PostgresSourceConfig generatePostgresConfig( if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); } - // Ensure ts_usec is present (required by PostgresOffset) - if (!offsetMap.containsKey(SourceInfo.TIMESTAMP_USEC_KEY)) { - offsetMap.put(SourceInfo.TIMESTAMP_USEC_KEY, - String.valueOf(Conversions.toEpochMicros(Instant.MIN))); - } configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { // start from timestamp From 4a8183ad9d917cc6cbb6e505ca05244ae5fcae24 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 11:37:46 +0800 Subject: [PATCH 06/22] [fix](streaming-job) remove unreachable named mode branch in deserializeOffsetProperty Co-Authored-By: Claude Opus 4.6 (1M context) --- .../doris/job/offset/jdbc/JdbcSourceOffsetProvider.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 5b876a4baffa25..118cf6672ab1c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -365,14 +365,6 @@ public Offset deserializeOffsetProperty(String offset) { if (offset == null || offset.trim().isEmpty()) { return null; } - // Named modes: stored in sourceProperties.offset, CDC client reads it directly. - // Return a placeholder JdbcOffset so validateOffset() passes. - if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset)) { - return new JdbcOffset(Collections.singletonList(new BinlogSplit())); - } // JSON format: {"file":"binlog.000003","pos":154} or {"lsn":"123456"} if (DataSourceConfigValidator.isJsonOffset(offset)) { try { From a543e1bb835f14e866d1eef4440f11ce381a4a91 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 12:05:44 +0800 Subject: [PATCH 07/22] fix style --- .../cdcclient/source/reader/postgres/PostgresSourceReader.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index f1a0c572f24dcf..e45e7ce21f9186 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -208,7 +208,8 @@ private PostgresSourceConfig generatePostgresConfig( } else if (ConfigUtil.isJson(startupMode)) { Map offsetMap = ConfigUtil.toStringMap(startupMode); if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { - throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); + throw new RuntimeException( + "JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); } configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { From 651316bb635735585aceee17530c062c3e583b60 Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 18:48:49 +0800 Subject: [PATCH 08/22] [Improve](streaming-job) support specifying offset for StreamingInsertJob create and alter Co-Authored-By: Claude Opus 4.6 (1M context) --- .../streaming/DataSourceConfigValidator.java | 54 +++- .../insert/streaming/StreamingInsertJob.java | 13 +- .../doris/job/offset/jdbc/JdbcOffset.java | 7 +- .../offset/jdbc/JdbcSourceOffsetProvider.java | 24 +- .../trees/plans/commands/AlterJobCommand.java | 3 +- .../plans/commands/CreateJobCommand.java | 3 +- .../reader/JdbcIncrementalSourceReader.java | 4 +- .../reader/postgres/PostgresSourceReader.java | 11 +- ..._streaming_mysql_job_special_offset.groovy | 229 ++++++++++++++ ...mysql_job_special_offset_restart_fe.groovy | 140 +++++++++ ...reaming_postgres_job_special_offset.groovy | 296 ++++++++++++++++++ 11 files changed, 764 insertions(+), 20 deletions(-) create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java index b75e202b1a842e..3700b80e6a6e30 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -18,14 +18,19 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.common.DataSourceType; import org.apache.doris.nereids.trees.plans.commands.LoadCommand; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; import java.util.Map; import java.util.Set; public class DataSourceConfigValidator { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final Set ALLOW_SOURCE_KEYS = Sets.newHashSet( DataSourceConfigKeys.JDBC_URL, DataSourceConfigKeys.USER, @@ -51,7 +56,7 @@ public class DataSourceConfigValidator { private static final String TABLE_LEVEL_PREFIX = DataSourceConfigKeys.TABLE + "."; - public static void validateSource(Map input) throws IllegalArgumentException { + public static void validateSource(Map input, String dataSourceType) throws IllegalArgumentException { for (Map.Entry entry : input.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); @@ -79,7 +84,7 @@ public static void validateSource(Map input) throws IllegalArgum throw new IllegalArgumentException("Unexpected key: '" + key + "'"); } - if (!isValidValue(key, value)) { + if (!isValidValue(key, value, dataSourceType)) { throw new IllegalArgumentException("Invalid value for key '" + key + "': " + value); } } @@ -103,18 +108,51 @@ public static void validateTarget(Map input) throws IllegalArgum } } - private static boolean isValidValue(String key, String value) { + private static boolean isValidValue(String key, String value, String dataSourceType) { if (value == null || value.isEmpty()) { return false; } - if (key.equals(DataSourceConfigKeys.OFFSET) - && !(value.equals(DataSourceConfigKeys.OFFSET_INITIAL) - || value.equals(DataSourceConfigKeys.OFFSET_LATEST) - || value.equals(DataSourceConfigKeys.OFFSET_SNAPSHOT))) { - return false; + if (key.equals(DataSourceConfigKeys.OFFSET)) { + return isValidOffset(value, dataSourceType); } return true; } + /** + * Check if the offset value is valid for the given data source type. + * Supported: initial, snapshot, latest, JSON binlog/lsn position. + * earliest is only supported for MySQL. + */ + public static boolean isValidOffset(String offset, String dataSourceType) { + if (offset == null || offset.isEmpty()) { + return false; + } + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset)) { + return true; + } + // earliest only for MySQL + if (DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset)) { + return DataSourceType.MYSQL.name().equalsIgnoreCase(dataSourceType); + } + if (isJsonOffset(offset)) { + return true; + } + return false; + } + + public static boolean isJsonOffset(String offset) { + if (offset == null || offset.trim().isEmpty()) { + return false; + } + try { + JsonNode node = OBJECT_MAPPER.readTree(offset); + return node.isObject(); + } catch (Exception e) { + return false; + } + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 01e994e2e9da56..695e9c33195950 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -306,9 +306,8 @@ private void initInsertJob() { this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); this.offsetProvider.ensureInitialized(getJobId(), originTvfProps); this.offsetProvider.initOnCreate(); - // validate offset props, only for s3 cause s3 tvf no offset prop - if (jobProperties.getOffsetProperty() != null - && S3TableValuedFunction.NAME.equalsIgnoreCase(tvfType)) { + // validate offset props + if (jobProperties.getOffsetProperty() != null) { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } @@ -783,14 +782,18 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { */ private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); - if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty()) - && S3TableValuedFunction.NAME.equalsIgnoreCase(this.tvfType)) { + if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); this.offsetProviderPersist = offsetProvider.getPersistInfo(); if (Config.isCloudMode()) { resetCloudProgress(offset); } + // For FROM...TO path, also update sourceProperties so the CDC client + // uses the new offset when building the next task. + if (this.sourceProperties != null) { + this.sourceProperties.put(DataSourceConfigKeys.OFFSET, inputStreamProps.getOffsetProperty()); + } } this.properties.putAll(inputProperties); this.jobProperties = new StreamingJobProperties(this.properties); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index dea2c244d6d78c..faf39dbe30899d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -48,7 +48,10 @@ public class JdbcOffset implements Offset { @Override public String toSerializedJson() { - return null; + if (splits == null || splits.isEmpty()) { + return null; + } + return new Gson().toJson(splits); } @Override @@ -58,7 +61,7 @@ public boolean isEmpty() { @Override public boolean isValidOffset() { - return false; + return splits != null && !splits.isEmpty(); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index f6bbdcc5b56386..75a8c880684e74 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -29,6 +29,7 @@ import org.apache.doris.job.cdc.split.SnapshotSplit; import org.apache.doris.job.common.DataSourceType; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.DataSourceConfigValidator; import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; import org.apache.doris.job.offset.Offset; @@ -360,7 +361,28 @@ public Offset deserializeOffset(String offset) { @Override public Offset deserializeOffsetProperty(String offset) { - // no need cause cdc_stream has offset property + if (offset == null || offset.trim().isEmpty()) { + return null; + } + // Named modes: stored in sourceProperties.offset, CDC client reads it directly. + // Return a placeholder JdbcOffset so validateOffset() passes. + if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset) + || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset)) { + return new JdbcOffset(Collections.singletonList(new BinlogSplit())); + } + // JSON format: {"file":"binlog.000003","pos":154} or {"lsn":"123456"} + if (DataSourceConfigValidator.isJsonOffset(offset)) { + try { + Map offsetMap = objectMapper.readValue(offset, + new TypeReference>() {}); + return new JdbcOffset(Collections.singletonList(new BinlogSplit(offsetMap))); + } catch (Exception e) { + log.warn("Failed to parse JSON offset: {}", offset, e); + return null; + } + } return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index fc63710f93ddf1..c0455ab6c1c504 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -166,7 +166,8 @@ private void validate() throws Exception { boolean sourcePropModified = isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); if (sourcePropModified) { - DataSourceConfigValidator.validateSource(this.getSourceProperties()); + DataSourceConfigValidator.validateSource(this.getSourceProperties(), + streamingJob.getSourceType()); checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java index 1640143d277f01..ff278293e8216d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/CreateJobCommand.java @@ -74,7 +74,8 @@ private void validate() throws JobException { } if (StringUtils.isNotEmpty(createJobInfo.getSourceType())) { - DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties()); + DataSourceConfigValidator.validateSource(createJobInfo.getSourceProperties(), + createJobInfo.getSourceType()); DataSourceConfigValidator.validateTarget(createJobInfo.getTargetProperties()); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java index 3d9167d6124f1d..3c0e6a8d001f8c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/JdbcIncrementalSourceReader.java @@ -720,8 +720,10 @@ private Offset getStartOffsetFromConfig(JdbcSourceConfig sourceConfig) { case EARLIEST_OFFSET: startingOffset = createInitialOffset(); break; - case TIMESTAMP: case SPECIFIC_OFFSETS: + startingOffset = createOffset(startupOptions.getOffset()); + break; + case TIMESTAMP: case COMMITTED_OFFSETS: default: throw new IllegalStateException( diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index b6d28510613c94..e4fcd44b35015c 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -206,7 +206,16 @@ private PostgresSourceConfig generatePostgresConfig( } else if (DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(startupMode)) { configFactory.startupOptions(StartupOptions.latest()); } else if (ConfigUtil.isJson(startupMode)) { - throw new RuntimeException("Unsupported json offset " + startupMode); + Map offsetMap = ConfigUtil.toStringMap(startupMode); + if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { + throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); + } + // Ensure ts_usec is present (required by PostgresOffset) + if (!offsetMap.containsKey(SourceInfo.TIMESTAMP_USEC_KEY)) { + offsetMap.put(SourceInfo.TIMESTAMP_USEC_KEY, + String.valueOf(Conversions.toEpochMicros(Instant.MIN))); + } + configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { // start from timestamp Long ts = Long.parseLong(startupMode); diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy new file mode 100644 index 00000000000000..da9331e524feed --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -0,0 +1,229 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_mysql_job_special_offset" + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_mysql_tbl" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `id` int NOT NULL, + `name` varchar(100) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // ===== Test 1: offset = earliest, verify data synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "earliest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" + assert rows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 2: offset = latest, then insert new data, verify synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + // wait for job to be running + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // insert new data after job started with latest offset + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 1 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3: ALTER JOB to change offset ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // pause, then alter offset to initial + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to initial, snapshot data should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 3 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 4: invalid offset format ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "not_valid_offset" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 5: JSON binlog offset, verify data synced ===== + // Get current binlog position, insert data after it, then create job from that position + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Current binlog position: file=${binlogFile}, pos=${binlogPos}") + // insert data after this binlog position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (10, 'specific1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (11, 'specific2')""" + } + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("Using JSON offset: ${offsetJson}") + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = '${offsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def specificRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (10, 11) order by id""" + log.info("specificRows: " + specificRows) + assert specificRows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // cleanup MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + } + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy new file mode 100644 index 00000000000000..9b89d43334fa0f --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy @@ -0,0 +1,140 @@ +// 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. + +import org.apache.doris.regression.suite.ClusterOptions +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_mysql_job_special_offset_restart_fe", "docker,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_mysql_job_special_offset_restart_fe" + def options = new ClusterOptions() + options.setFeNum(1) + // run in cloud and not cloud + options.cloudMode = null + + docker(options) { + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_restart_tbl" + def mysqlDb = "test_cdc_db" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare MySQL source table and get binlog position + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + sql """CREATE TABLE ${mysqlDb}.${table1} ( + `id` int NOT NULL, + `name` varchar(100) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + // get current binlog position + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Binlog position: file=${binlogFile}, pos=${binlogPos}") + // insert data after this position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // create job with JSON binlog offset + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("Creating job with offset: ${offsetJson}") + sql """CREATE JOB ${jobName} + ON STREAMING + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = '${offsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + + // wait for data synced + try { + Awaitility.await().atMost(300, SECONDS) + .pollInterval(1, SECONDS).until( + { + def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ + log.info("jobSuccendCount: " + jobSuccendCount) + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + } + ) + } catch (Exception ex) { + def showjob = sql """select * from jobs("type"="insert") where Name='${jobName}'""" + def showtask = sql """select * from tasks("type"="insert") where JobName='${jobName}'""" + log.info("show job: " + showjob) + log.info("show task: " + showtask) + throw ex; + } + + def jobInfoBefore = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoBefore: " + jobInfoBefore) + assert jobInfoBefore.get(0).get(1) == "RUNNING" + + // Restart FE + cluster.restartFrontends() + sleep(60000) + context.reconnectFe() + + // check job is consistent after restart + def jobInfoAfter = sql """ + select loadStatistic, status, currentOffset from jobs("type"="insert") where Name='${jobName}' + """ + log.info("jobInfoAfter: " + jobInfoAfter) + assert jobInfoAfter.get(0).get(1) == "RUNNING" + assert jobInfoAfter.get(0).get(2) == jobInfoBefore.get(0).get(2) + + // insert more data and verify job still works after restart + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 3 + }) + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" + } + } + } +} diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy new file mode 100644 index 00000000000000..6244b62b98b1d6 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -0,0 +1,296 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_docker,external_docker_pg,nondatalake") { + def jobName = "test_streaming_pg_job_special_offset" + def currentDb = (sql "select database()")[0][0] + def table1 = "special_offset_pg_tbl" + def pgDB = "postgres" + def pgSchema = "cdc_test" + def pgUser = "postgres" + def pgPassword = "123456" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String pg_port = context.config.otherConfigs.get("pg_14_port"); + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/postgresql-42.5.0.jar" + + // prepare PG source table + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" + sql """CREATE TABLE ${pgDB}.${pgSchema}.${table1} ( + "id" int, + "name" varchar(100), + PRIMARY KEY ("id") + )""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (1, 'alice'), (2, 'bob')""" + } + + // ===== Test 1: offset = initial, verify data synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" + assert rows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 2: offset = latest, then insert new data, verify synced ===== + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // insert new data after job started with latest offset + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (3, 'charlie')""" + } + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 1 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3: ALTER JOB to change offset ===== + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DELETE FROM ${pgDB}.${pgSchema}.${table1} WHERE id = 3""" + } + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + }) + // pause, then alter offset to initial + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to initial, snapshot data should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 4: earliest should fail for PG ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "earliest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 5: invalid offset format ===== + test { + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "not_valid" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "Invalid value for key 'offset'" + } + + // ===== Test 6: JSON LSN offset via ALTER, verify data synced ===== + // Step 1: Create job with initial to establish replication slot and sync snapshot + sql """CREATE JOB ${jobName} + ON STREAMING + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "initial" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + + // Step 2: Get current WAL LSN, then insert new data + def currentLsn = "" + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + def lsnResult = sql """SELECT pg_current_wal_lsn()::text""" + def lsnStr = lsnResult[0][0].toString() + // Convert PG LSN format (e.g. "0/1A3B4C0") to numeric + def parts = lsnStr.split("/") + def high = Long.parseLong(parts[0], 16) + def low = Long.parseLong(parts[1], 16) + currentLsn = String.valueOf((high << 32) + low) + log.info("Current WAL LSN: ${lsnStr} -> numeric: ${currentLsn}") + // insert new data after this LSN + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (20, 'lsn_test1')""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (21, 'lsn_test2')""" + } + + // Step 3: PAUSE -> ALTER with JSON LSN -> RESUME + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + def lsnOffsetJson = """{"lsn":"${currentLsn}"}""" + log.info("Using JSON LSN offset: ${lsnOffsetJson}") + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = '${lsnOffsetJson}' + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + sql "RESUME JOB where jobname = '${jobName}'" + + // Step 4: Verify new data synced + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" + return result[0][0] >= 2 + }) + def lsnRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (20, 21) order by id""" + log.info("lsnRows: " + lsnRows) + assert lsnRows.size() == 2 + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // cleanup PG source table + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" + } + } +} From 3537e0ec4a3f2c823c6cf9d586748f630db93dca Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 19:14:42 +0800 Subject: [PATCH 09/22] [Improve](streaming-job) ALTER CDC offset only supports JSON, add TVF alter offset test - CDC ALTER only allows JSON specific offset (reject named modes like initial/latest) - ALTER offset uses PROPERTIES('offset'='{"file":"xxx","pos":"yyy"}') syntax - Update regression cases to use PROPERTIES for ALTER - Add cdc_stream TVF ALTER offset regression test Co-Authored-By: Claude Opus 4.6 (1M context) --- .../insert/streaming/StreamingInsertJob.java | 14 +- ..._streaming_mysql_job_special_offset.groovy | 51 +++++- ...reaming_postgres_job_special_offset.groovy | 51 +----- ...g_job_cdc_stream_mysql_alter_offset.groovy | 155 ++++++++++++++++++ 4 files changed, 217 insertions(+), 54 deletions(-) create mode 100644 regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 695e9c33195950..7611e94b8a0c64 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -783,17 +783,21 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { + // For CDC jobs, ALTER only supports JSON specific offset (e.g. binlog position or LSN), + // named modes like initial/latest/snapshot are only valid at CREATE time. + if (offsetProvider instanceof JdbcSourceOffsetProvider + && !DataSourceConfigValidator.isJsonOffset(inputStreamProps.getOffsetProperty())) { + throw new AnalysisException( + "ALTER JOB for CDC only supports JSON specific offset, " + + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " + + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); + } Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); this.offsetProviderPersist = offsetProvider.getPersistInfo(); if (Config.isCloudMode()) { resetCloudProgress(offset); } - // For FROM...TO path, also update sourceProperties so the CDC client - // uses the new offset when building the next task. - if (this.sourceProperties != null) { - this.sourceProperties.put(DataSourceConfigKeys.OFFSET, inputStreamProps.getOffsetProperty()); - } } this.properties.putAll(inputProperties); this.jobProperties = new StreamingJobProperties(this.properties); diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index da9331e524feed..c7757e3094f4cc 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -108,7 +108,19 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER JOB to change offset ===== + // ===== Test 3: ALTER JOB with JSON binlog offset via PROPERTIES ===== + // Get current binlog position, then create job with initial + def alterBinlogFile = "" + def alterBinlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + alterBinlogFile = masterStatus[0][0] + alterBinlogPos = masterStatus[0][1].toString() + log.info("ALTER test binlog position: file=${alterBinlogFile}, pos=${alterBinlogPos}") + // insert data after this position + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (20, 'alter_test1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (21, 'alter_test2')""" + } sql """CREATE JOB ${jobName} ON STREAMING FROM MYSQL ( @@ -129,13 +141,29 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) - // pause, then alter offset to initial + // pause, then alter offset to specific binlog position via PROPERTIES sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus[0][0] == "PAUSED" }) + def alterOffsetJson = """{"file":"${alterBinlogFile}","pos":"${alterBinlogPos}"}""" + log.info("ALTER offset: ${alterOffsetJson}") sql """ALTER JOB ${jobName} + PROPERTIES('offset' = '${alterOffsetJson}') + """ + sql "RESUME JOB where jobname = '${jobName}'" + // after alter to specific binlog position, data inserted after that position should sync + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" + return result[0][0] >= 2 + }) + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${table1} force""" + + // ===== Test 3b: ALTER with named mode should fail for CDC ===== + sql """CREATE JOB ${jobName} + ON STREAMING FROM MYSQL ( "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", "driver_url" = "${driver_url}", @@ -150,12 +178,21 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc "table.create.properties.replication_num" = "1" ) """ - sql "RESUME JOB where jobname = '${jobName}'" - // after alter to initial, snapshot data should sync - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 3 + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + test { + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = 'initial') + """ + exception "ALTER JOB for CDC only supports JSON specific offset" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 6244b62b98b1d6..2dc84e6648c984 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -111,10 +111,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER JOB to change offset ===== - connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { - sql """DELETE FROM ${pgDB}.${pgSchema}.${table1} WHERE id = 3""" - } + // ===== Test 3: ALTER with named mode should fail for CDC ===== sql """CREATE JOB ${jobName} ON STREAMING FROM POSTGRES ( @@ -126,7 +123,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc "database" = "${pgDB}", "schema" = "${pgSchema}", "include_tables" = "${table1}", - "offset" = "latest" + "offset" = "initial" ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" @@ -136,34 +133,17 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) - // pause, then alter offset to initial sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus[0][0] == "PAUSED" }) - sql """ALTER JOB ${jobName} - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - sql "RESUME JOB where jobname = '${jobName}'" - // after alter to initial, snapshot data should sync - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) + test { + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = 'initial') + """ + exception "ALTER JOB for CDC only supports JSON specific offset" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" @@ -260,20 +240,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc def lsnOffsetJson = """{"lsn":"${currentLsn}"}""" log.info("Using JSON LSN offset: ${lsnOffsetJson}") sql """ALTER JOB ${jobName} - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = '${lsnOffsetJson}' - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) + PROPERTIES('offset' = '${lsnOffsetJson}') """ sql "RESUME JOB where jobname = '${jobName}'" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy new file mode 100644 index 00000000000000..5158da072f8636 --- /dev/null +++ b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy @@ -0,0 +1,155 @@ +// 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. + + +import org.awaitility.Awaitility + +import static java.util.concurrent.TimeUnit.SECONDS + +/** + * Test ALTER JOB with JSON binlog offset for cdc_stream TVF path. + * + * Scenario: + * 1. Create job with initial offset, wait for snapshot sync. + * 2. Get current binlog position, insert new data. + * 3. PAUSE -> ALTER with JSON binlog offset via PROPERTIES -> RESUME. + * 4. Verify new data synced. + */ +suite("test_streaming_job_cdc_stream_mysql_alter_offset", "p0,external,mysql,external_docker,external_docker_mysql,nondatalake") { + def jobName = "test_streaming_job_cdc_stream_mysql_alter_offset_name" + def currentDb = (sql "select database()")[0][0] + def dorisTable = "cdc_stream_alter_offset_tbl" + def mysqlDb = "test_cdc_db" + def mysqlTable = "cdc_stream_alter_offset_src" + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${dorisTable} force""" + + sql """ + CREATE TABLE IF NOT EXISTS ${currentDb}.${dorisTable} ( + `id` int NULL, + `name` varchar(200) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS AUTO + PROPERTIES ("replication_allocation" = "tag.location.default: 1") + """ + + String enabled = context.config.otherConfigs.get("enableJdbcTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String mysql_port = context.config.otherConfigs.get("mysql_57_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + String s3_endpoint = getS3Endpoint() + String bucket = getS3BucketName() + String driver_url = "https://${bucket}.${s3_endpoint}/regression/jdbc_driver/mysql-connector-j-8.4.0.jar" + + // prepare source table with snapshot data + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """CREATE DATABASE IF NOT EXISTS ${mysqlDb}""" + sql """DROP TABLE IF EXISTS ${mysqlDb}.${mysqlTable}""" + sql """CREATE TABLE ${mysqlDb}.${mysqlTable} ( + `id` int NOT NULL, + `name` varchar(200) DEFAULT NULL, + PRIMARY KEY (`id`) + ) ENGINE=InnoDB""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (1, 'alice'), (2, 'bob')""" + } + + // Step 1: Create job with initial offset via cdc_stream TVF + sql """ + CREATE JOB ${jobName} + ON STREAMING DO INSERT INTO ${currentDb}.${dorisTable} (id, name) + SELECT id, name FROM cdc_stream( + "type" = "mysql", + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "table" = "${mysqlTable}", + "offset" = "initial" + ) + """ + + // wait for snapshot sync + try { + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ + def cnt = sql """select SucceedTaskCount from jobs("type"="insert") where Name='${jobName}' and ExecuteType='STREAMING'""" + log.info("SucceedTaskCount: " + cnt) + cnt.size() == 1 && (cnt.get(0).get(0) as int) >= 1 + }) + } catch (Exception ex) { + log.info("job: " + (sql """select * from jobs("type"="insert") where Name='${jobName}'""")) + log.info("tasks: " + (sql """select * from tasks("type"="insert") where JobName='${jobName}'""")) + throw ex + } + + // verify snapshot data + Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable}""" + return result[0][0] >= 2 + }) + + // Step 2: Get current binlog position and insert new data + def binlogFile = "" + def binlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("Binlog position for ALTER: file=${binlogFile}, pos=${binlogPos}") + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (10, 'alter_tvf_1')""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (11, 'alter_tvf_2')""" + } + + // Step 3: PAUSE -> ALTER with JSON binlog offset -> RESUME + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("ALTER TVF job offset: ${offsetJson}") + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = '${offsetJson}') + """ + + // verify currentOffset changed in show jobs + def jobInfo = sql """select currentOffset from jobs("type"="insert") where Name='${jobName}'""" + log.info("currentOffset after ALTER: " + jobInfo[0][0]) + + sql "RESUME JOB where jobname = '${jobName}'" + + // Step 4: Verify new data synced + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11)""" + return result[0][0] >= 2 + }) + def alterRows = sql """SELECT * FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11) order by id""" + log.info("alterRows: " + alterRows) + assert alterRows.size() == 2 + + sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" + sql """drop table if exists ${currentDb}.${dorisTable} force""" + + // cleanup MySQL source table + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + sql """DROP TABLE IF EXISTS ${mysqlDb}.${mysqlTable}""" + } + } +} From 0feb39c1693a167088dda9e4224474cc9d9b091a Mon Sep 17 00:00:00 2001 From: wudi Date: Tue, 14 Apr 2026 19:15:36 +0800 Subject: [PATCH 10/22] [fix](streaming-job) fix checkstyle: line length and unused import Co-Authored-By: Claude Opus 4.6 (1M context) --- .../extensions/insert/streaming/DataSourceConfigValidator.java | 3 ++- .../job/extensions/insert/streaming/StreamingInsertJob.java | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java index 3700b80e6a6e30..5ef98668cbab12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidator.java @@ -56,7 +56,8 @@ public class DataSourceConfigValidator { private static final String TABLE_LEVEL_PREFIX = DataSourceConfigKeys.TABLE + "."; - public static void validateSource(Map input, String dataSourceType) throws IllegalArgumentException { + public static void validateSource(Map input, + String dataSourceType) throws IllegalArgumentException { for (Map.Entry entry : input.entrySet()) { String key = entry.getKey(); String value = entry.getValue(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 7611e94b8a0c64..6cc33c4c1eb736 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -73,7 +73,6 @@ import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; -import org.apache.doris.tablefunction.S3TableValuedFunction; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; From 70eaeff3c9fd6e68cf9018098148343802d81d14 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 10:14:23 +0800 Subject: [PATCH 11/22] [fix](streaming-job) fix AlterJobCommand compile error: use getDataSourceType().name() Co-Authored-By: Claude Opus 4.6 (1M context) --- .../doris/nereids/trees/plans/commands/AlterJobCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index c0455ab6c1c504..029bf932e5cc82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -167,7 +167,7 @@ private void validate() throws Exception { isPropertiesModified(streamingJob.getSourceProperties(), this.getSourceProperties()); if (sourcePropModified) { DataSourceConfigValidator.validateSource(this.getSourceProperties(), - streamingJob.getSourceType()); + streamingJob.getDataSourceType().name()); checkUnmodifiableSourceProperties(streamingJob.getSourceProperties()); } From d605d89ab042a749ba9a136265d3602fa84edffd Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 11:27:32 +0800 Subject: [PATCH 12/22] [Improve](streaming-job) address review: validate ALTER offset in provider, fix serialization - Add validateAlterOffset() to SourceOffsetProvider interface, CDC impl rejects non-JSON offset, S3 impl allows any format (default no-op) - StreamingInsertJob delegates ALTER offset validation to provider - AlterJobCommand.validateProps calls validateAlterOffset before validateOffset - Restore S3-only guard in initInsertJob for PROPERTIES offset - Fix JdbcOffset.toSerializedJson() to produce flat format compatible with deserializeOffset(): [{"split_id":"binlog-split","file":"xxx","pos":"yyy"}] - Remove unnecessary ts_usec padding in PostgresSourceReader JSON offset - Remove instanceof check in modifyPropertiesInternal (validation moved to AlterJobCommand) Co-Authored-By: Claude Opus 4.6 (1M context) --- .../insert/streaming/StreamingInsertJob.java | 26 +++++++++++-------- .../job/offset/SourceOffsetProvider.java | 7 +++++ .../doris/job/offset/jdbc/JdbcOffset.java | 13 +++++++++- .../offset/jdbc/JdbcSourceOffsetProvider.java | 11 ++++++++ .../trees/plans/commands/AlterJobCommand.java | 5 ++-- .../reader/postgres/PostgresSourceReader.java | 5 ---- 6 files changed, 47 insertions(+), 20 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 6cc33c4c1eb736..8727b9363ca470 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -73,6 +73,7 @@ import org.apache.doris.qe.ShowResultSetMetaData; import org.apache.doris.rpc.RpcException; import org.apache.doris.service.FrontendOptions; +import org.apache.doris.tablefunction.S3TableValuedFunction; import org.apache.doris.thrift.TCell; import org.apache.doris.thrift.TRow; import org.apache.doris.transaction.TransactionException; @@ -305,8 +306,9 @@ private void initInsertJob() { this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); this.offsetProvider.ensureInitialized(getJobId(), originTvfProps); this.offsetProvider.initOnCreate(); - // validate offset props - if (jobProperties.getOffsetProperty() != null) { + // validate offset props, only for s3 cause s3 tvf no offset prop + if (jobProperties.getOffsetProperty() != null + && S3TableValuedFunction.NAME.equalsIgnoreCase(tvfType)) { Offset offset = validateOffset(jobProperties.getOffsetProperty()); this.offsetProvider.updateOffset(offset); } @@ -357,6 +359,17 @@ public void initLogicalPlan(boolean regen) { } } + /** + * Validate the offset format for ALTER JOB, delegating to the provider. + */ + public void validateAlterOffset(String offset) throws AnalysisException { + try { + offsetProvider.validateAlterOffset(offset); + } catch (Exception ex) { + throw new AnalysisException(ex.getMessage()); + } + } + /** * Check whether Offset can be serialized into the corresponding data source * */ @@ -782,15 +795,6 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { private void modifyPropertiesInternal(Map inputProperties) throws AnalysisException, JobException { StreamingJobProperties inputStreamProps = new StreamingJobProperties(inputProperties); if (StringUtils.isNotEmpty(inputStreamProps.getOffsetProperty())) { - // For CDC jobs, ALTER only supports JSON specific offset (e.g. binlog position or LSN), - // named modes like initial/latest/snapshot are only valid at CREATE time. - if (offsetProvider instanceof JdbcSourceOffsetProvider - && !DataSourceConfigValidator.isJsonOffset(inputStreamProps.getOffsetProperty())) { - throw new AnalysisException( - "ALTER JOB for CDC only supports JSON specific offset, " - + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " - + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); - } Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); this.offsetProviderPersist = offsetProvider.getPersistInfo(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java index 03ac09b1e3fe06..7823f7be1ad19c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProvider.java @@ -114,6 +114,13 @@ default void initOnCreate() throws JobException {} */ Offset deserializeOffsetProperty(String offset); + /** + * Validate the offset format for ALTER JOB. + * Each provider defines its own rules (e.g. CDC only allows JSON specific offset). + */ + default void validateAlterOffset(String offset) throws Exception { + } + /** * Replaying OffsetProvider is currently only required by JDBC. * diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java index faf39dbe30899d..d6a29900a6c524 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcOffset.java @@ -32,6 +32,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.MapUtils; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,7 +52,17 @@ public String toSerializedJson() { if (splits == null || splits.isEmpty()) { return null; } - return new Gson().toJson(splits); + // Serialize to the same flat format that deserializeOffset() expects: + // [{"split_id":"binlog-split","file":"xxx","pos":"yyy"}] + Preconditions.checkState(splits.size() == 1 && splits.get(0) instanceof BinlogSplit, + "toSerializedJson only supports single BinlogSplit"); + BinlogSplit binlog = (BinlogSplit) splits.get(0); + Map map = new HashMap<>(); + map.put(JdbcSourceOffsetProvider.SPLIT_ID, binlog.getSplitId()); + if (binlog.getStartingOffset() != null) { + map.putAll(binlog.getStartingOffset()); + } + return new Gson().toJson(Collections.singletonList(map)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 75a8c880684e74..5b876a4baffa25 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -18,6 +18,7 @@ package org.apache.doris.job.offset.jdbc; import org.apache.doris.catalog.Env; +import org.apache.doris.common.AnalysisException; import org.apache.doris.httpv2.entity.ResponseBody; import org.apache.doris.httpv2.rest.RestApiStatusCode; import org.apache.doris.job.cdc.DataSourceConfigKeys; @@ -386,6 +387,16 @@ public Offset deserializeOffsetProperty(String offset) { return null; } + @Override + public void validateAlterOffset(String offset) throws Exception { + if (!DataSourceConfigValidator.isJsonOffset(offset)) { + throw new AnalysisException( + "ALTER JOB for CDC only supports JSON specific offset, " + + "e.g. '{\"file\":\"binlog.000001\",\"pos\":\"154\"}' for MySQL " + + "or '{\"lsn\":\"12345678\"}' for PostgreSQL"); + } + } + /** * Replay snapshot splits if needed */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 029bf932e5cc82..7974c7751e03ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -218,9 +218,8 @@ private void checkUnmodifiableSourceProperties(Map originSourceP private void validateProps(StreamingInsertJob streamingJob) throws AnalysisException { StreamingJobProperties jobProperties = new StreamingJobProperties(properties); jobProperties.validate(); - // from to job no need valiate offset in job properties - if (streamingJob.getDataSourceType() == null - && jobProperties.getOffsetProperty() != null) { + if (jobProperties.getOffsetProperty() != null) { + streamingJob.validateAlterOffset(jobProperties.getOffsetProperty()); streamingJob.validateOffset(jobProperties.getOffsetProperty()); } } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index e4fcd44b35015c..f1a0c572f24dcf 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -210,11 +210,6 @@ private PostgresSourceConfig generatePostgresConfig( if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); } - // Ensure ts_usec is present (required by PostgresOffset) - if (!offsetMap.containsKey(SourceInfo.TIMESTAMP_USEC_KEY)) { - offsetMap.put(SourceInfo.TIMESTAMP_USEC_KEY, - String.valueOf(Conversions.toEpochMicros(Instant.MIN))); - } configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { // start from timestamp From 528a075a7d0e5c16ee2e300fa59183ddfe07e81e Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 11:37:46 +0800 Subject: [PATCH 13/22] [fix](streaming-job) remove unreachable named mode branch in deserializeOffsetProperty Co-Authored-By: Claude Opus 4.6 (1M context) --- .../doris/job/offset/jdbc/JdbcSourceOffsetProvider.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java index 5b876a4baffa25..118cf6672ab1c7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/jdbc/JdbcSourceOffsetProvider.java @@ -365,14 +365,6 @@ public Offset deserializeOffsetProperty(String offset) { if (offset == null || offset.trim().isEmpty()) { return null; } - // Named modes: stored in sourceProperties.offset, CDC client reads it directly. - // Return a placeholder JdbcOffset so validateOffset() passes. - if (DataSourceConfigKeys.OFFSET_INITIAL.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_SNAPSHOT.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_EARLIEST.equalsIgnoreCase(offset) - || DataSourceConfigKeys.OFFSET_LATEST.equalsIgnoreCase(offset)) { - return new JdbcOffset(Collections.singletonList(new BinlogSplit())); - } // JSON format: {"file":"binlog.000003","pos":154} or {"lsn":"123456"} if (DataSourceConfigValidator.isJsonOffset(offset)) { try { From b7c7a15cc9cddf5227fd5934d383a2879989a207 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 12:05:44 +0800 Subject: [PATCH 14/22] fix style --- .../cdcclient/source/reader/postgres/PostgresSourceReader.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index f1a0c572f24dcf..e45e7ce21f9186 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -208,7 +208,8 @@ private PostgresSourceConfig generatePostgresConfig( } else if (ConfigUtil.isJson(startupMode)) { Map offsetMap = ConfigUtil.toStringMap(startupMode); if (offsetMap == null || !offsetMap.containsKey(SourceInfo.LSN_KEY)) { - throw new RuntimeException("JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); + throw new RuntimeException( + "JSON offset for PostgreSQL must contain 'lsn' key, got: " + startupMode); } configFactory.startupOptions(StartupOptions.specificOffset(offsetMap)); } else if (ConfigUtil.is13Timestamp(startupMode)) { From 98ca5b94e9336a71d9c356a239bfffa9521a5c88 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 18:32:34 +0800 Subject: [PATCH 15/22] [Improve](streaming-job) fix ALTER offset: supplement kind/ts_usec, improve test cases - MySQL BE: supplement kind=SPECIFIC for ALTER offset in createBinlogSplit, add WARN log for fallback to config/earliest offset - PG BE: supplement ts_usec=0 in createOffset for ALTER offset - FE: sync offsetProviderPersist after ALTER offset update, remove debug logs from hasMoreDataToConsume - Improve test cases: verify ALTER offset truly takes effect by checking data before/after the offset mark position - Add golden files for qt_select assertions Co-Authored-By: Claude Opus 4.6 (1M context) --- .../insert/streaming/StreamingInsertJob.java | 2 + .../reader/mysql/MySqlSourceReader.java | 23 ++- .../reader/postgres/PostgresSourceReader.java | 7 + ...est_streaming_mysql_job_special_offset.out | 11 ++ ..._streaming_postgres_job_special_offset.out | 13 ++ ..._streaming_mysql_job_special_offset.groovy | 153 +++++------------ ...reaming_postgres_job_special_offset.groovy | 155 +++++++----------- ...g_job_cdc_stream_mysql_alter_offset.groovy | 44 ++--- 8 files changed, 176 insertions(+), 232 deletions(-) create mode 100644 regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.out create mode 100644 regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.out diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 8727b9363ca470..16c6e75cc864c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -798,6 +798,8 @@ private void modifyPropertiesInternal(Map inputProperties) throw Offset offset = validateOffset(inputStreamProps.getOffsetProperty()); this.offsetProvider.updateOffset(offset); this.offsetProviderPersist = offsetProvider.getPersistInfo(); + log.info("modifyPropertiesInternal: offset updated to {}, job {}", + inputStreamProps.getOffsetProperty(), getJobId()); if (Config.isCloudMode()) { resetCloudProgress(offset); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java index 14f6a6de1d4a94..291b84329f7e01 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/mysql/MySqlSourceReader.java @@ -48,6 +48,7 @@ import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; +import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetKind; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffsetUtils; import org.apache.flink.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlBinlogSplit; @@ -638,18 +639,30 @@ private Tuple2 createBinlogSplit( } BinlogOffset startOffset; - BinlogOffset lastOffset = - new BinlogOffset( - binlogSplit.getStartingOffset() == null - ? new HashMap<>() - : binlogSplit.getStartingOffset()); + Map lastOffsetMap = + binlogSplit.getStartingOffset() == null + ? new HashMap<>() + : new HashMap<>(binlogSplit.getStartingOffset()); + // ALTER offset may not contain "kind", supplement it for Flink CDC + if (lastOffsetMap.containsKey(BinlogOffset.BINLOG_FILENAME_OFFSET_KEY) + && !lastOffsetMap.containsKey(BinlogOffset.OFFSET_KIND_KEY)) { + lastOffsetMap.put(BinlogOffset.OFFSET_KIND_KEY, BinlogOffsetKind.SPECIFIC.name()); + } + BinlogOffset lastOffset = new BinlogOffset(lastOffsetMap); if (minOffsetFinishSplits != null && lastOffset.getOffsetKind() == null) { startOffset = minOffsetFinishSplits; } else if (lastOffset.getOffsetKind() != null && lastOffset.getFilename() != null) { startOffset = lastOffset; } else if (offsetConfig != null) { + LOG.warn( + "Falling back to startup config offset {}, meta offset was: {}", + offsetConfig, + lastOffsetMap); startOffset = offsetConfig; } else { + LOG.warn( + "No valid offset found, falling back to earliest. meta offset was: {}", + lastOffsetMap); startOffset = BinlogOffset.ofEarliest(); } diff --git a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java index e45e7ce21f9186..1556f1313e6580 100644 --- a/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java +++ b/fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/source/reader/postgres/PostgresSourceReader.java @@ -295,6 +295,13 @@ protected OffsetFactory getOffsetFactory() { @Override protected Offset createOffset(Map offset) { + // ALTER offset may only contain lsn, supplement ts_usec for PostgresOffsetContext.Loader + if (offset.containsKey(SourceInfo.LSN_KEY) + && !offset.containsKey(SourceInfo.TIMESTAMP_USEC_KEY)) { + Map supplemented = new HashMap<>(offset); + supplemented.put(SourceInfo.TIMESTAMP_USEC_KEY, "0"); + return PostgresOffset.of(supplemented); + } return PostgresOffset.of(offset); } diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.out new file mode 100644 index 00000000000000..2fc5212efaf13b --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.out @@ -0,0 +1,11 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_after_create -- +10 specific1 +11 specific2 + +-- !select_after_alter -- +10 specific1 +11 specific2 +20 alter1 +21 alter2 + diff --git a/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.out b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.out new file mode 100644 index 00000000000000..d14feb4178551f --- /dev/null +++ b/regression-test/data/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select_after_create -- +1 alice +2 bob +3 charlie + +-- !select_after_alter -- +1 alice +2 bob +3 charlie +30 after_lsn +31 after_lsn + diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index c7757e3094f4cc..d298b93fe170b2 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -49,33 +49,7 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc sql """INSERT INTO ${mysqlDb}.${table1} VALUES (1, 'alice'), (2, 'bob')""" } - // ===== Test 1: offset = earliest, verify data synced ===== - sql """CREATE JOB ${jobName} - ON STREAMING - FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "user" = "root", - "password" = "123456", - "database" = "${mysqlDb}", - "include_tables" = "${table1}", - "offset" = "earliest" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) - def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" - assert rows.size() == 2 - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" - - // ===== Test 2: offset = latest, then insert new data, verify synced ===== + // ===== Test 1: offset = latest, then insert new data, verify synced ===== sql """CREATE JOB ${jobName} ON STREAMING FROM MYSQL ( @@ -108,19 +82,31 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER JOB with JSON binlog offset via PROPERTIES ===== - // Get current binlog position, then create job with initial - def alterBinlogFile = "" - def alterBinlogPos = "" + // ===== Test 3: CREATE with JSON binlog offset, then ALTER to earlier offset ===== + // Pre-create a DUPLICATE KEY table so duplicate rows from re-consuming are visible + sql """ + CREATE TABLE IF NOT EXISTS ${currentDb}.${table1} ( + `id` int NULL, + `name` varchar(100) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS AUTO + PROPERTIES ("replication_allocation" = "tag.location.default: 1") + """ + + // Step 1: Get binlog position, insert data, create job from that position + def binlogFile = "" + def binlogPos = "" connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { def masterStatus = sql """SHOW MASTER STATUS""" - alterBinlogFile = masterStatus[0][0] - alterBinlogPos = masterStatus[0][1].toString() - log.info("ALTER test binlog position: file=${alterBinlogFile}, pos=${alterBinlogPos}") - // insert data after this position - sql """INSERT INTO ${mysqlDb}.${table1} VALUES (20, 'alter_test1')""" - sql """INSERT INTO ${mysqlDb}.${table1} VALUES (21, 'alter_test2')""" + binlogFile = masterStatus[0][0] + binlogPos = masterStatus[0][1].toString() + log.info("CREATE binlog position: file=${binlogFile}, pos=${binlogPos}") + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (10, 'specific1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (11, 'specific2')""" } + def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" + log.info("CREATE with JSON offset: ${offsetJson}") sql """CREATE JOB ${jobName} ON STREAMING FROM MYSQL ( @@ -131,57 +117,49 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc "password" = "123456", "database" = "${mysqlDb}", "include_tables" = "${table1}", - "offset" = "latest" + "offset" = '${offsetJson}' ) TO DATABASE ${currentDb} ( "table.create.properties.replication_num" = "1" ) """ - Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ - def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" - return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 }) - // pause, then alter offset to specific binlog position via PROPERTIES + // Verify data after CREATE with specific offset + qt_select_after_create """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ + + // Step 2: Get a new binlog position (different from CREATE), insert data, ALTER to it + def alterBinlogFile = "" + def alterBinlogPos = "" + connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + def masterStatus = sql """SHOW MASTER STATUS""" + alterBinlogFile = masterStatus[0][0] + alterBinlogPos = masterStatus[0][1].toString() + log.info("ALTER binlog position: file=${alterBinlogFile}, pos=${alterBinlogPos}") + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (20, 'alter1')""" + sql """INSERT INTO ${mysqlDb}.${table1} VALUES (21, 'alter2')""" + } sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus[0][0] == "PAUSED" }) def alterOffsetJson = """{"file":"${alterBinlogFile}","pos":"${alterBinlogPos}"}""" - log.info("ALTER offset: ${alterOffsetJson}") + log.info("ALTER to new offset: ${alterOffsetJson}") sql """ALTER JOB ${jobName} PROPERTIES('offset' = '${alterOffsetJson}') """ sql "RESUME JOB where jobname = '${jobName}'" - // after alter to specific binlog position, data inserted after that position should sync + // After ALTER to new position, id 20,21 should be synced Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" return result[0][0] >= 2 }) - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" + qt_select_after_alter """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ - // ===== Test 3b: ALTER with named mode should fail for CDC ===== - sql """CREATE JOB ${jobName} - ON STREAMING - FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "user" = "root", - "password" = "123456", - "database" = "${mysqlDb}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ - def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" - return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" - }) + // Step 3: ALTER with named mode should fail for CDC sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" @@ -217,47 +195,6 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc exception "Invalid value for key 'offset'" } - // ===== Test 5: JSON binlog offset, verify data synced ===== - // Get current binlog position, insert data after it, then create job from that position - def binlogFile = "" - def binlogPos = "" - connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { - def masterStatus = sql """SHOW MASTER STATUS""" - binlogFile = masterStatus[0][0] - binlogPos = masterStatus[0][1].toString() - log.info("Current binlog position: file=${binlogFile}, pos=${binlogPos}") - // insert data after this binlog position - sql """INSERT INTO ${mysqlDb}.${table1} VALUES (10, 'specific1')""" - sql """INSERT INTO ${mysqlDb}.${table1} VALUES (11, 'specific2')""" - } - def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" - log.info("Using JSON offset: ${offsetJson}") - sql """CREATE JOB ${jobName} - ON STREAMING - FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "user" = "root", - "password" = "123456", - "database" = "${mysqlDb}", - "include_tables" = "${table1}", - "offset" = '${offsetJson}' - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) - def specificRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (10, 11) order by id""" - log.info("specificRows: " + specificRows) - assert specificRows.size() == 2 - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" - // cleanup MySQL source table connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { sql """DROP TABLE IF EXISTS ${mysqlDb}.${table1}""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 2dc84e6648c984..feb7a89c146c3c 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -51,34 +51,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (1, 'alice'), (2, 'bob')""" } - // ===== Test 1: offset = initial, verify data synced ===== - sql """CREATE JOB ${jobName} - ON STREAMING - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) - def rows = sql """SELECT * FROM ${currentDb}.${table1} order by id""" - assert rows.size() == 2 - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" - - // ===== Test 2: offset = latest, then insert new data, verify synced ===== + // ===== Test 1: offset = latest, then insert new data, verify synced ===== sql """CREATE JOB ${jobName} ON STREAMING FROM POSTGRES ( @@ -100,7 +73,6 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" }) - // insert new data after job started with latest offset connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (3, 'charlie')""" } @@ -111,7 +83,19 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 3: ALTER with named mode should fail for CDC ===== + // ===== Test 2: CREATE with initial, then ALTER with JSON LSN offset ===== + // Pre-create a DUPLICATE KEY table so duplicate rows from re-consuming are visible + sql """ + CREATE TABLE IF NOT EXISTS ${currentDb}.${table1} ( + `id` int NULL, + `name` varchar(100) NULL + ) ENGINE=OLAP + DUPLICATE KEY(`id`) + DISTRIBUTED BY HASH(`id`) BUCKETS AUTO + PROPERTIES ("replication_allocation" = "tag.location.default: 1") + """ + + // Step 1: Create job with initial to establish replication slot and sync snapshot sql """CREATE JOB ${jobName} ON STREAMING FROM POSTGRES ( @@ -129,10 +113,49 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc "table.create.properties.replication_num" = "1" ) """ - Awaitility.await().atMost(60, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" + return result[0][0] >= 2 + }) + qt_select_after_create """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ + + // Step 2: PAUSE, insert data before and after a LSN mark, ALTER to that LSN + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" - return jobStatus.size() == 1 && jobStatus[0][0] == "RUNNING" + return jobStatus[0][0] == "PAUSED" + }) + def alterLsn = "" + connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { + // insert data BEFORE the LSN mark + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (20, 'before_lsn')""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (21, 'before_lsn')""" + // record LSN mark + def lsnResult = sql """SELECT pg_current_wal_lsn()::text""" + def lsnStr = lsnResult[0][0].toString() + def parts = lsnStr.split("/") + def high = Long.parseLong(parts[0], 16) + def low = Long.parseLong(parts[1], 16) + alterLsn = String.valueOf((high << 32) + low) + log.info("ALTER LSN mark: ${lsnStr} -> numeric: ${alterLsn}") + // insert data AFTER the LSN mark + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (30, 'after_lsn')""" + sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (31, 'after_lsn')""" + } + def alterOffsetJson = """{"lsn":"${alterLsn}"}""" + log.info("ALTER to LSN: ${alterOffsetJson}") + sql """ALTER JOB ${jobName} + PROPERTIES('offset' = '${alterOffsetJson}') + """ + sql "RESUME JOB where jobname = '${jobName}'" + // After ALTER to LSN mark, only data AFTER that LSN (id 30,31) should be synced + Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (30, 31)""" + return result[0][0] >= 2 }) + qt_select_after_alter """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ + + // Step 3: ALTER with named mode should fail for CDC sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" @@ -147,7 +170,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" - // ===== Test 4: earliest should fail for PG ===== + // ===== Test 3: earliest should fail for PG ===== test { sql """CREATE JOB ${jobName} ON STREAMING @@ -169,7 +192,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc exception "Invalid value for key 'offset'" } - // ===== Test 5: invalid offset format ===== + // ===== Test 4: invalid offset format ===== test { sql """CREATE JOB ${jobName} ON STREAMING @@ -191,70 +214,6 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc exception "Invalid value for key 'offset'" } - // ===== Test 6: JSON LSN offset via ALTER, verify data synced ===== - // Step 1: Create job with initial to establish replication slot and sync snapshot - sql """CREATE JOB ${jobName} - ON STREAMING - FROM POSTGRES ( - "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", - "driver_url" = "${driver_url}", - "driver_class" = "org.postgresql.Driver", - "user" = "${pgUser}", - "password" = "${pgPassword}", - "database" = "${pgDB}", - "schema" = "${pgSchema}", - "include_tables" = "${table1}", - "offset" = "initial" - ) - TO DATABASE ${currentDb} ( - "table.create.properties.replication_num" = "1" - ) - """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" - return result[0][0] >= 2 - }) - - // Step 2: Get current WAL LSN, then insert new data - def currentLsn = "" - connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { - def lsnResult = sql """SELECT pg_current_wal_lsn()::text""" - def lsnStr = lsnResult[0][0].toString() - // Convert PG LSN format (e.g. "0/1A3B4C0") to numeric - def parts = lsnStr.split("/") - def high = Long.parseLong(parts[0], 16) - def low = Long.parseLong(parts[1], 16) - currentLsn = String.valueOf((high << 32) + low) - log.info("Current WAL LSN: ${lsnStr} -> numeric: ${currentLsn}") - // insert new data after this LSN - sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (20, 'lsn_test1')""" - sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (21, 'lsn_test2')""" - } - - // Step 3: PAUSE -> ALTER with JSON LSN -> RESUME - sql "PAUSE JOB where jobname = '${jobName}'" - Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ - def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" - return jobStatus[0][0] == "PAUSED" - }) - def lsnOffsetJson = """{"lsn":"${currentLsn}"}""" - log.info("Using JSON LSN offset: ${lsnOffsetJson}") - sql """ALTER JOB ${jobName} - PROPERTIES('offset' = '${lsnOffsetJson}') - """ - sql "RESUME JOB where jobname = '${jobName}'" - - // Step 4: Verify new data synced - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" - return result[0][0] >= 2 - }) - def lsnRows = sql """SELECT * FROM ${currentDb}.${table1} WHERE id IN (20, 21) order by id""" - log.info("lsnRows: " + lsnRows) - assert lsnRows.size() == 2 - sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" - sql """drop table if exists ${currentDb}.${table1} force""" - // cleanup PG source table connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { sql """DROP TABLE IF EXISTS ${pgDB}.${pgSchema}.${table1}""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy index 5158da072f8636..b3de4e92a771cc 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy @@ -105,44 +105,46 @@ suite("test_streaming_job_cdc_stream_mysql_alter_offset", "p0,external,mysql,ext return result[0][0] >= 2 }) - // Step 2: Get current binlog position and insert new data + // Step 2: PAUSE, insert data before and after a binlog mark, ALTER to that mark + sql "PAUSE JOB where jobname = '${jobName}'" + Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ + def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" + return jobStatus[0][0] == "PAUSED" + }) def binlogFile = "" def binlogPos = "" connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { + // insert data BEFORE the binlog mark + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (10, 'before_mark')""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (11, 'before_mark')""" + // record binlog mark def masterStatus = sql """SHOW MASTER STATUS""" binlogFile = masterStatus[0][0] binlogPos = masterStatus[0][1].toString() - log.info("Binlog position for ALTER: file=${binlogFile}, pos=${binlogPos}") - sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (10, 'alter_tvf_1')""" - sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (11, 'alter_tvf_2')""" + log.info("Binlog mark for ALTER: file=${binlogFile}, pos=${binlogPos}") + // insert data AFTER the binlog mark + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (20, 'after_mark')""" + sql """INSERT INTO ${mysqlDb}.${mysqlTable} VALUES (21, 'after_mark')""" } - - // Step 3: PAUSE -> ALTER with JSON binlog offset -> RESUME - sql "PAUSE JOB where jobname = '${jobName}'" - Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ - def jobStatus = sql """select status from jobs("type"="insert") where Name='${jobName}'""" - return jobStatus[0][0] == "PAUSED" - }) def offsetJson = """{"file":"${binlogFile}","pos":"${binlogPos}"}""" log.info("ALTER TVF job offset: ${offsetJson}") sql """ALTER JOB ${jobName} PROPERTIES('offset' = '${offsetJson}') """ - - // verify currentOffset changed in show jobs - def jobInfo = sql """select currentOffset from jobs("type"="insert") where Name='${jobName}'""" - log.info("currentOffset after ALTER: " + jobInfo[0][0]) - sql "RESUME JOB where jobname = '${jobName}'" - // Step 4: Verify new data synced + // Step 3: Verify only data AFTER the mark (id 20,21) is synced Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ - def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11)""" + def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable} WHERE id IN (20, 21)""" return result[0][0] >= 2 }) - def alterRows = sql """SELECT * FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11) order by id""" - log.info("alterRows: " + alterRows) - assert alterRows.size() == 2 + def afterMarkRows = sql """SELECT * FROM ${currentDb}.${dorisTable} WHERE id IN (20, 21) order by id""" + log.info("afterMarkRows: " + afterMarkRows) + assert afterMarkRows.size() == 2 + // id 10,11 (before mark) should NOT be synced + def beforeMarkRows = sql """SELECT * FROM ${currentDb}.${dorisTable} WHERE id IN (10, 11)""" + log.info("beforeMarkRows: " + beforeMarkRows) + assert beforeMarkRows.size() == 0 sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${dorisTable} force""" From 85a43d1e2222170f8d181c55ee8268597e7a303d Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 19:28:41 +0800 Subject: [PATCH 16/22] [fix](streaming-job) relax restart FE test to wait for 1 succeed task instead of 2 Co-Authored-By: Claude Opus 4.6 (1M context) --- .../test_streaming_mysql_job_special_offset_restart_fe.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy index 9b89d43334fa0f..9ac882a4493be2 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy @@ -90,7 +90,7 @@ suite("test_streaming_mysql_job_special_offset_restart_fe", "docker,mysql,extern { def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ log.info("jobSuccendCount: " + jobSuccendCount) - jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) + jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) } ) } catch (Exception ex) { From daf67bae2a74188d00e6d30b7fa2ce2ed0820586 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 19:39:28 +0800 Subject: [PATCH 17/22] [fix](streaming-job) reject offset change in ALTER source properties, guide user to PROPERTIES Co-Authored-By: Claude Opus 4.6 (1M context) --- .../nereids/trees/plans/commands/AlterJobCommand.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 7974c7751e03ed..6482d8046a9a5e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -213,6 +213,14 @@ private void checkUnmodifiableSourceProperties(Map originSourceP sourceProperties.get(DataSourceConfigKeys.EXCLUDE_TABLES)), "The exclude_tables property cannot be modified in ALTER JOB"); } + + if (sourceProperties.containsKey(DataSourceConfigKeys.OFFSET) + && !Objects.equals(originSourceProperties.get(DataSourceConfigKeys.OFFSET), + sourceProperties.get(DataSourceConfigKeys.OFFSET))) { + throw new AnalysisException( + "The offset in source properties cannot be modified in ALTER JOB. " + + "Use PROPERTIES('offset'='{...}') to alter offset"); + } } private void validateProps(StreamingInsertJob streamingJob) throws AnalysisException { From 1349e844c687b73f9b5f0386e6b66e2cb3920dd9 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 19:41:23 +0800 Subject: [PATCH 18/22] [fix](streaming-job) reject offset change in ALTER source properties, add test cases Co-Authored-By: Claude Opus 4.6 (1M context) --- ..._streaming_mysql_job_special_offset.groovy | 19 ++++++++++++++++++ ...reaming_postgres_job_special_offset.groovy | 20 +++++++++++++++++++ 2 files changed, 39 insertions(+) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index d298b93fe170b2..175fd09892d5d2 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -171,6 +171,25 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc """ exception "ALTER JOB for CDC only supports JSON specific offset" } + // ALTER offset via source properties should be rejected + test { + sql """ALTER JOB ${jobName} + FROM MYSQL ( + "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", + "driver_url" = "${driver_url}", + "driver_class" = "com.mysql.cj.jdbc.Driver", + "user" = "root", + "password" = "123456", + "database" = "${mysqlDb}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The offset in source properties cannot be modified in ALTER JOB" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index feb7a89c146c3c..9c3cb9b8ecaa25 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -167,6 +167,26 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc """ exception "ALTER JOB for CDC only supports JSON specific offset" } + // ALTER offset via source properties should be rejected + test { + sql """ALTER JOB ${jobName} + FROM POSTGRES ( + "jdbc_url" = "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}", + "driver_url" = "${driver_url}", + "driver_class" = "org.postgresql.Driver", + "user" = "${pgUser}", + "password" = "${pgPassword}", + "database" = "${pgDB}", + "schema" = "${pgSchema}", + "include_tables" = "${table1}", + "offset" = "latest" + ) + TO DATABASE ${currentDb} ( + "table.create.properties.replication_num" = "1" + ) + """ + exception "The offset in source properties cannot be modified in ALTER JOB" + } sql """DROP JOB IF EXISTS where jobname = '${jobName}'""" sql """drop table if exists ${currentDb}.${table1} force""" From 7836643ab929e093613ca3737ef2c5a785f3a045 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 20:02:49 +0800 Subject: [PATCH 19/22] [Improve](streaming-job) reject sourceProperties offset in ALTER, increase PG await timeout Co-Authored-By: Claude Opus 4.6 (1M context) --- .../nereids/trees/plans/commands/AlterJobCommand.java | 8 ++++---- .../cdc/test_streaming_postgres_job_special_offset.groovy | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java index 6482d8046a9a5e..1e8cde06bf91b9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/AlterJobCommand.java @@ -214,10 +214,10 @@ private void checkUnmodifiableSourceProperties(Map originSourceP "The exclude_tables property cannot be modified in ALTER JOB"); } - if (sourceProperties.containsKey(DataSourceConfigKeys.OFFSET) - && !Objects.equals(originSourceProperties.get(DataSourceConfigKeys.OFFSET), - sourceProperties.get(DataSourceConfigKeys.OFFSET))) { - throw new AnalysisException( + if (sourceProperties.containsKey(DataSourceConfigKeys.OFFSET)) { + Preconditions.checkArgument(Objects.equals( + originSourceProperties.get(DataSourceConfigKeys.OFFSET), + sourceProperties.get(DataSourceConfigKeys.OFFSET)), "The offset in source properties cannot be modified in ALTER JOB. " + "Use PROPERTIES('offset'='{...}') to alter offset"); } diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 9c3cb9b8ecaa25..97bb50374aef46 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -149,7 +149,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc """ sql "RESUME JOB where jobname = '${jobName}'" // After ALTER to LSN mark, only data AFTER that LSN (id 30,31) should be synced - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(240, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (30, 31)""" return result[0][0] >= 2 }) From 2b7aae67c2fffc0b9b67d6e6e6c53cdfde8900b0 Mon Sep 17 00:00:00 2001 From: wudi Date: Wed, 15 Apr 2026 20:04:17 +0800 Subject: [PATCH 20/22] [fix](streaming-job) increase all await timeout to 300s for stability Co-Authored-By: Claude Opus 4.6 (1M context) --- .../cdc/test_streaming_mysql_job_special_offset.groovy | 6 +++--- ...est_streaming_mysql_job_special_offset_restart_fe.groovy | 2 +- .../cdc/test_streaming_postgres_job_special_offset.groovy | 6 +++--- .../test_streaming_job_cdc_stream_mysql_alter_offset.groovy | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index 175fd09892d5d2..06a0b853e0e3eb 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -75,7 +75,7 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" } - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" return result[0][0] >= 1 }) @@ -123,7 +123,7 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc "table.create.properties.replication_num" = "1" ) """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" return result[0][0] >= 2 }) @@ -153,7 +153,7 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc """ sql "RESUME JOB where jobname = '${jobName}'" // After ALTER to new position, id 20,21 should be synced - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (20, 21)""" return result[0][0] >= 2 }) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy index 9ac882a4493be2..ad6c79e93cddd5 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset_restart_fe.groovy @@ -124,7 +124,7 @@ suite("test_streaming_mysql_job_special_offset_restart_fe", "docker,mysql,extern connect("root", "123456", "jdbc:mysql://${externalEnvIp}:${mysql_port}") { sql """INSERT INTO ${mysqlDb}.${table1} VALUES (3, 'charlie')""" } - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" return result[0][0] >= 3 }) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 97bb50374aef46..12cf974ce950d0 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -76,7 +76,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc connect("${pgUser}", "${pgPassword}", "jdbc:postgresql://${externalEnvIp}:${pg_port}/${pgDB}") { sql """INSERT INTO ${pgDB}.${pgSchema}.${table1} VALUES (3, 'charlie')""" } - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" return result[0][0] >= 1 }) @@ -113,7 +113,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc "table.create.properties.replication_num" = "1" ) """ - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1}""" return result[0][0] >= 2 }) @@ -149,7 +149,7 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc """ sql "RESUME JOB where jobname = '${jobName}'" // After ALTER to LSN mark, only data AFTER that LSN (id 30,31) should be synced - Awaitility.await().atMost(240, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${table1} WHERE id IN (30, 31)""" return result[0][0] >= 2 }) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy index b3de4e92a771cc..7093be81e36aeb 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy @@ -134,7 +134,7 @@ suite("test_streaming_job_cdc_stream_mysql_alter_offset", "p0,external,mysql,ext sql "RESUME JOB where jobname = '${jobName}'" // Step 3: Verify only data AFTER the mark (id 20,21) is synced - Awaitility.await().atMost(120, SECONDS).pollInterval(2, SECONDS).until({ + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ def result = sql """SELECT count(*) FROM ${currentDb}.${dorisTable} WHERE id IN (20, 21)""" return result[0][0] >= 2 }) From 0811de17f972a0e0ee2eefe93f8f33d6ed977f25 Mon Sep 17 00:00:00 2001 From: wudi Date: Thu, 16 Apr 2026 11:36:33 +0800 Subject: [PATCH 21/22] fix case --- .../test_streaming_mysql_job_create_alter.groovy | 14 -------------- .../cdc/test_streaming_mysql_job_priv.groovy | 10 ++-------- .../test_streaming_mysql_job_special_offset.groovy | 7 +++++++ ...st_streaming_postgres_job_special_offset.groovy | 7 +++++++ ...eaming_job_cdc_stream_mysql_alter_offset.groovy | 7 +++++++ 5 files changed, 23 insertions(+), 22 deletions(-) diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy index 3cff806161640d..0b3a2d2eee85e9 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_create_alter.groovy @@ -405,19 +405,6 @@ suite("test_streaming_mysql_job_create_alter", "p0,external,mysql,external_docke exception "target database can't be modified" } - sql """ALTER JOB ${jobName} - FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", - "user" = "root", - "password" = "123456", - "database" = "${mysqlDb}", - "include_tables" = "${table1}", - "offset" = "latest" - ) - TO DATABASE ${currentDb}""" - def jobInfoOrigin = sql """ select CurrentOffset,LoadStatistic from jobs("type"="insert") where Name='${jobName}' """ @@ -459,7 +446,6 @@ suite("test_streaming_mysql_job_create_alter", "p0,external,mysql,external_docke assert jobInfoCurrent.get(0).get(1) == jobInfoOrigin.get(0).get(1) assert jobInfoCurrent.get(0).get(2).contains("\"max_interval\":\"5\"") assert jobInfoCurrent.get(0).get(2).contains("\"__source_subtype\":\"aws_rds_mysql\"") - assert jobInfoCurrent.get(0).get(3).contains("latest") sql """ DROP JOB IF EXISTS where jobname = '${jobName}' diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy index 4909e3d0469c0c..540b96b04638d9 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_priv.groovy @@ -128,7 +128,7 @@ suite("test_streaming_mysql_job_priv", "p0,external,mysql,external_docker,extern def jobSuccendCount = sql """ select SucceedTaskCount from jobs("type"="insert") where Name = '${jobName}' and ExecuteType='STREAMING' """ log.info("jobSuccendCount: " + jobSuccendCount) // check job status and succeed task count larger than 2 - jobSuccendCount.size() == 1 && '1' <= jobSuccendCount.get(0).get(0) + jobSuccendCount.size() == 1 && '2' <= jobSuccendCount.get(0).get(0) } ) } catch (Exception ex) { @@ -155,14 +155,8 @@ suite("test_streaming_mysql_job_priv", "p0,external,mysql,external_docker,extern sql """ALTER JOB ${jobName} FROM MYSQL ( - "jdbc_url" = "jdbc:mysql://${externalEnvIp}:${mysql_port}?allowPublicKeyRetrieval=true&useSSL=false", - "driver_url" = "${driver_url}", - "driver_class" = "com.mysql.cj.jdbc.Driver", "user" = "${newMysqlUser}", - "password" = "test123", - "database" = "${mysqlDb}", - "include_tables" = "${tableName}", - "offset" = "latest" + "password" = "test123" ) TO DATABASE ${currentDb}""" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy index 06a0b853e0e3eb..3c988239225a65 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_mysql_job_special_offset.groovy @@ -130,6 +130,13 @@ suite("test_streaming_mysql_job_special_offset", "p0,external,mysql,external_doc // Verify data after CREATE with specific offset qt_select_after_create """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ + // Wait for current task to complete (commit offset successfully) before PAUSE, + // otherwise PAUSE may race with a running task and cause commit offset failure. + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ + def cnt = sql """select SucceedTaskCount from jobs("type"="insert") where Name='${jobName}' and ExecuteType='STREAMING'""" + return cnt.size() == 1 && (cnt.get(0).get(0) as int) >= 2 + }) + // Step 2: Get a new binlog position (different from CREATE), insert data, ALTER to it def alterBinlogFile = "" def alterBinlogPos = "" diff --git a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy index 12cf974ce950d0..1b64f111859aea 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/test_streaming_postgres_job_special_offset.groovy @@ -119,6 +119,13 @@ suite("test_streaming_postgres_job_special_offset", "p0,external,pg,external_doc }) qt_select_after_create """ SELECT * FROM ${currentDb}.${table1} ORDER BY id """ + // Wait for current task to complete (commit offset successfully) before PAUSE, + // otherwise PAUSE may race with a running task and cause commit offset failure. + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ + def cnt = sql """select SucceedTaskCount from jobs("type"="insert") where Name='${jobName}' and ExecuteType='STREAMING'""" + return cnt.size() == 1 && (cnt.get(0).get(0) as int) >= 2 + }) + // Step 2: PAUSE, insert data before and after a LSN mark, ALTER to that LSN sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ diff --git a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy index 7093be81e36aeb..db674f305dfe63 100644 --- a/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy +++ b/regression-test/suites/job_p0/streaming_job/cdc/tvf/test_streaming_job_cdc_stream_mysql_alter_offset.groovy @@ -105,6 +105,13 @@ suite("test_streaming_job_cdc_stream_mysql_alter_offset", "p0,external,mysql,ext return result[0][0] >= 2 }) + // Wait for current task to complete (commit offset successfully) before PAUSE, + // otherwise PAUSE may race with a running task and cause commit offset failure. + Awaitility.await().atMost(300, SECONDS).pollInterval(2, SECONDS).until({ + def cnt = sql """select SucceedTaskCount from jobs("type"="insert") where Name='${jobName}' and ExecuteType='STREAMING'""" + return cnt.size() == 1 && (cnt.get(0).get(0) as int) >= 2 + }) + // Step 2: PAUSE, insert data before and after a binlog mark, ALTER to that mark sql "PAUSE JOB where jobname = '${jobName}'" Awaitility.await().atMost(30, SECONDS).pollInterval(1, SECONDS).until({ From c7b1dcd39c79806bd99d667531328744c2caea45 Mon Sep 17 00:00:00 2001 From: wudi Date: Fri, 24 Apr 2026 17:33:14 +0800 Subject: [PATCH 22/22] [fix](streaming-job) fix DataSourceConfigValidatorTest after signature change validateSource now takes (Map, String dataSourceType). The test file was added upstream after this PR started changing the signature, so after the merge its six single-arg calls no longer compile. Pass DataSourceType.POSTGRES.name() since every test covers PG slot_name / publication_name validation. --- .../streaming/DataSourceConfigValidatorTest.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidatorTest.java index 50fd2ece020edd..2f71b7664a832d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/job/extensions/insert/streaming/DataSourceConfigValidatorTest.java @@ -18,6 +18,7 @@ package org.apache.doris.job.extensions.insert.streaming; import org.apache.doris.job.cdc.DataSourceConfigKeys; +import org.apache.doris.job.common.DataSourceType; import org.junit.Assert; import org.junit.Test; @@ -36,7 +37,7 @@ public void testSlotNameAndPublicationNameAllowed() { props.put(DataSourceConfigKeys.SLOT_NAME, "my_custom_slot"); props.put(DataSourceConfigKeys.PUBLICATION_NAME, "my_custom_pub"); // Should not throw - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); } @Test @@ -44,7 +45,7 @@ public void testSlotNameAndPublicationNameNotRequired() { Map props = new HashMap<>(); props.put(DataSourceConfigKeys.JDBC_URL, "jdbc:postgresql://localhost:5432/db"); // Should not throw without slot_name and publication_name - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); } @Test @@ -90,7 +91,7 @@ public void testSlotNameRejectsInvalidPgIdentifiers() { props.put(DataSourceConfigKeys.JDBC_URL, "jdbc:postgresql://localhost:5432/db"); props.put(DataSourceConfigKeys.SLOT_NAME, invalid); try { - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); Assert.fail("Expected IllegalArgumentException for slot_name='" + invalid + "'"); } catch (IllegalArgumentException expected) { // ok @@ -106,7 +107,7 @@ public void testPublicationNameRejectsInvalidPgIdentifiers() { props.put(DataSourceConfigKeys.JDBC_URL, "jdbc:postgresql://localhost:5432/db"); props.put(DataSourceConfigKeys.PUBLICATION_NAME, invalid); try { - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); Assert.fail("Expected IllegalArgumentException for publication_name='" + invalid + "'"); } catch (IllegalArgumentException expected) { // ok @@ -124,7 +125,7 @@ public void testSlotNameRejectsOverlongIdentifier() { props.put(DataSourceConfigKeys.JDBC_URL, "jdbc:postgresql://localhost:5432/db"); props.put(DataSourceConfigKeys.SLOT_NAME, sb.toString()); try { - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); Assert.fail("Expected IllegalArgumentException for slot_name exceeding " + PG_MAX_IDENTIFIER_LENGTH + " chars"); } catch (IllegalArgumentException expected) { @@ -139,7 +140,7 @@ public void testSlotNameAcceptsValidPgIdentifiers() { Map props = new HashMap<>(); props.put(DataSourceConfigKeys.JDBC_URL, "jdbc:postgresql://localhost:5432/db"); props.put(DataSourceConfigKeys.SLOT_NAME, valid); - DataSourceConfigValidator.validateSource(props); + DataSourceConfigValidator.validateSource(props, DataSourceType.POSTGRES.name()); } } }