From 05fc70bb3f875b7a2c4c5c89fb2e270bee4df226 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 17 Dec 2025 15:45:02 +0800 Subject: [PATCH 01/20] update --- maintainer/replica/replication_span.go | 16 ++- maintainer/replica/replication_span_test.go | 49 ++++++++ .../syncpoint_move_table/conf/changefeed.toml | 3 + .../conf/diff_config_part1.toml | 24 ++++ .../conf/diff_config_part2.toml | 7 ++ .../syncpoint_move_table/run.sh | 115 ++++++++++++++++++ 6 files changed, 213 insertions(+), 1 deletion(-) create mode 100644 tests/integration_tests/syncpoint_move_table/conf/changefeed.toml create mode 100644 tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml create mode 100644 tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml create mode 100644 tests/integration_tests/syncpoint_move_table/run.sh diff --git a/maintainer/replica/replication_span.go b/maintainer/replica/replication_span.go index 6e86f086db..1924616b9d 100644 --- a/maintainer/replica/replication_span.go +++ b/maintainer/replica/replication_span.go @@ -229,6 +229,20 @@ func (r *SpanReplication) GetGroupID() replica.GroupID { } func (r *SpanReplication) NewAddDispatcherMessage(server node.ID) *messaging.TargetMessage { + // When a syncpoint is in-flight (WAITING/WRITING), the maintainer will force the span checkpoint to BlockTs-1 + // when handling block status. If we start a moved/recreated dispatcher from that forced checkpoint, it may re-scan + // and re-apply events with commitTs <= BlockTs, which can race with the syncpoint write and corrupt the snapshot + // semantics of that syncpoint. Starting from BlockTs avoids replaying those events, and is safe because the + // dispatcher can only enter the syncpoint barrier after all events with commitTs <= BlockTs have been pushed + // downstream. + startTs := r.status.Load().CheckpointTs + if state := r.blockState.Load(); state != nil && + state.IsBlocked && + state.IsSyncPoint && + (state.Stage == heartbeatpb.BlockStage_WAITING || state.Stage == heartbeatpb.BlockStage_WRITING) && + state.BlockTs > startTs { + startTs = state.BlockTs + } return messaging.NewSingleTargetMessage(server, messaging.HeartbeatCollectorTopic, &heartbeatpb.ScheduleDispatcherRequest{ @@ -237,7 +251,7 @@ func (r *SpanReplication) NewAddDispatcherMessage(server node.ID) *messaging.Tar DispatcherID: r.ID.ToPB(), SchemaID: r.schemaID, Span: r.Span, - StartTs: r.status.Load().CheckpointTs, + StartTs: startTs, Mode: r.GetMode(), }, ScheduleAction: heartbeatpb.ScheduleAction_Create, diff --git a/maintainer/replica/replication_span_test.go b/maintainer/replica/replication_span_test.go index 36419fabb0..93bf973421 100644 --- a/maintainer/replica/replication_span_test.go +++ b/maintainer/replica/replication_span_test.go @@ -53,6 +53,55 @@ func TestSpanReplication_NewAddDispatcherMessage(t *testing.T) { require.Equal(t, heartbeatpb.ScheduleAction_Create, req.ScheduleAction) require.Equal(t, replicaSet.ID.ToPB(), req.Config.DispatcherID) require.Equal(t, replicaSet.schemaID, req.Config.SchemaID) + require.Equal(t, uint64(10), req.Config.StartTs) +} + +func TestSpanReplication_NewAddDispatcherMessage_UseBlockTsForInFlightSyncPoint(t *testing.T) { + t.Parallel() + + replicaSet := NewSpanReplication(common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceNamme), common.NewDispatcherID(), 1, getTableSpanByID(4), 9, common.DefaultMode, false) + replicaSet.UpdateBlockState(heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + IsSyncPoint: true, + Stage: heartbeatpb.BlockStage_WAITING, + }) + + msg := replicaSet.NewAddDispatcherMessage("node1") + req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) + require.Equal(t, uint64(10), req.Config.StartTs) +} + +func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsAfterSyncPointDone(t *testing.T) { + t.Parallel() + + replicaSet := NewSpanReplication(common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceNamme), common.NewDispatcherID(), 1, getTableSpanByID(4), 20, common.DefaultMode, false) + replicaSet.UpdateBlockState(heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + IsSyncPoint: true, + Stage: heartbeatpb.BlockStage_DONE, + }) + + msg := replicaSet.NewAddDispatcherMessage("node1") + req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) + require.Equal(t, uint64(20), req.Config.StartTs) +} + +func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsForDDL(t *testing.T) { + t.Parallel() + + replicaSet := NewSpanReplication(common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceNamme), common.NewDispatcherID(), 1, getTableSpanByID(4), 9, common.DefaultMode, false) + replicaSet.UpdateBlockState(heartbeatpb.State{ + IsBlocked: true, + BlockTs: 10, + IsSyncPoint: false, + Stage: heartbeatpb.BlockStage_WAITING, + }) + + msg := replicaSet.NewAddDispatcherMessage("node1") + req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) + require.Equal(t, uint64(9), req.Config.StartTs) } // getTableSpanByID returns a mock TableSpan for testing diff --git a/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml b/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml new file mode 100644 index 0000000000..c21ae7ca60 --- /dev/null +++ b/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml @@ -0,0 +1,3 @@ +enable-sync-point = true +sync-point-interval = "30s" + diff --git a/tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml b/tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml new file mode 100644 index 0000000000..c1967b6bde --- /dev/null +++ b/tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml @@ -0,0 +1,24 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/syncpoint_move_table/sync_diff/output" + + source-instances = ["tidb0"] + + target-instance = "mysql1" + + target-check-tables = ["sp_move.*"] + +[data-sources] +[data-sources.tidb0] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + diff --git a/tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml b/tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml new file mode 100644 index 0000000000..235ea5a8e3 --- /dev/null +++ b/tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml @@ -0,0 +1,7 @@ + +[data-sources.mysql1] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh new file mode 100644 index 0000000000..cda7cf448f --- /dev/null +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -0,0 +1,115 @@ +#!/bin/bash +# +# This test verifies that moving a table dispatcher during an in-flight syncpoint does not +# restart the dispatcher from (syncpoint_ts - 1). Otherwise, the dispatcher may re-scan and +# re-apply events with commitTs <= syncpoint_ts while the table-trigger dispatcher is writing +# the syncpoint, which can break the snapshot consistency semantics. + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare + +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +DB_NAME="sp_move" +TABLE_NAME="t" +CHANGEFEED_ID="test" + +deployConfig() { + cat $CUR/conf/diff_config_part1.toml >$CUR/conf/diff_config.toml + echo "snapshot = \"$1\"" >>$CUR/conf/diff_config.toml + cat $CUR/conf/diff_config_part2.toml >>$CUR/conf/diff_config.toml + echo "snapshot = \"$2\"" >>$CUR/conf/diff_config.toml +} + +run() { + if [ "$SINK_TYPE" != "mysql" ]; then + echo "only mysql sink supports syncpoint record" + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + # This test contains `set global tidb_external_ts = ?`, which requires super privilege. + run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + do_retry 5 3 cdc_cli_changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" -c "$CHANGEFEED_ID" + + run_sql "DROP DATABASE IF EXISTS ${DB_NAME};" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE DATABASE ${DB_NAME};" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE DATABASE ${DB_NAME};" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} + + run_sql "CREATE TABLE ${DB_NAME}.${TABLE_NAME} (id INT PRIMARY KEY, v INT);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO ${DB_NAME}.${TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists "${DB_NAME}.${TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + + # Restart node1 to enable failpoints: + # - StopBalanceScheduler: keep the table on node1 until we explicitly move it. + # - BlockOrWaitBeforeWrite: block syncpoint writing on the table-trigger dispatcher. + cdc_pid_1=$(get_cdc_pid "$CDC_HOST" "$CDC_PORT") + kill_cdc_pid $cdc_pid_1 + cleanup_process $CDC_BINARY + + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(30000)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-1" --addr "127.0.0.1:8300" + + check_coordinator_and_maintainer "127.0.0.1:8300" "$CHANGEFEED_ID" 60 + + # Start node2 for moving the table. + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" + + # Keep generating DML to ensure a syncpoint is triggered. + for i in $(seq 10 50); do + run_sql "INSERT INTO ${DB_NAME}.${TABLE_NAME} (id, v) VALUES (${i}, ${i}) ON DUPLICATE KEY UPDATE v = v + 1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + sleep 1 + done + + # Wait for the table-trigger dispatcher to receive a syncpoint event and extract its commitTs. + ensure 30 "grep -q \"dispatcher receive sync point event\" $WORK_DIR/cdc0-1.log" + syncpoint_ts=$(grep "dispatcher receive sync point event" $WORK_DIR/cdc0-1.log | head -n 1 | grep -oE 'commitTs[^0-9]*[0-9]+' | head -n 1 | grep -oE '[0-9]+' || true) + if [ -z "$syncpoint_ts" ]; then + echo "failed to extract syncpoint commitTs from logs" + exit 1 + fi + echo "syncpoint_ts: $syncpoint_ts" + + # Ensure the table-trigger dispatcher has received the maintainer action to start writing this syncpoint. + ensure 30 "grep \"pending event get the action\" $WORK_DIR/cdc0-1.log | grep -Eq \"(pendingEventCommitTs[^0-9]*${syncpoint_ts}.*innerAction=0|innerAction=0.*pendingEventCommitTs[^0-9]*${syncpoint_ts})\"" + + table_id=$(get_table_id "$DB_NAME" "$TABLE_NAME") + move_table_with_retry "127.0.0.1:8301" $table_id "$CHANGEFEED_ID" 10 + + # The moved dispatcher must start from syncpoint_ts (not syncpoint_ts-1). + ensure 30 "grep \"new dispatcher created\" $WORK_DIR/cdc1.log | grep -q \"tableID: ${table_id}\"" + dispatcher_start_ts=$(grep "new dispatcher created" $WORK_DIR/cdc1.log | grep "tableID: ${table_id}" | tail -n 1 | grep -oE 'startTs[^0-9]*[0-9]+' | tail -n 1 | grep -oE '[0-9]+' || true) + if [ "$dispatcher_start_ts" != "$syncpoint_ts" ]; then + echo "unexpected dispatcher startTs, got: $dispatcher_start_ts, want: $syncpoint_ts" + exit 1 + fi + + # Wait until the syncpoint is written downstream, then validate snapshot consistency by sync_diff. + ensure 30 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" + secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) + echo "secondary_ts: $secondary_ts" + + deployConfig "$syncpoint_ts" "$secondary_ts" + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 60 + rm -f $CUR/conf/diff_config.toml + + export GO_FAILPOINTS='' + cleanup_process $CDC_BINARY +} + +trap 'stop_tidb_cluster; collect_logs $WORK_DIR' EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 30822884393b1421776b8fbb5821701962b854df Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 17 Dec 2025 16:36:07 +0800 Subject: [PATCH 02/20] update --- tests/integration_tests/syncpoint_move_table/run.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh index cda7cf448f..879caea581 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -42,7 +42,7 @@ run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - do_retry 5 3 cdc_cli_changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml" -c "$CHANGEFEED_ID" + do_retry 5 3 cdc_cli_changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" -c "$CHANGEFEED_ID" run_sql "DROP DATABASE IF EXISTS ${DB_NAME};" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE DATABASE ${DB_NAME};" ${UP_TIDB_HOST} ${UP_TIDB_PORT} @@ -59,11 +59,17 @@ run() { kill_cdc_pid $cdc_pid_1 cleanup_process $CDC_BINARY - export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(30000)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(90000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-1" --addr "127.0.0.1:8300" check_coordinator_and_maintainer "127.0.0.1:8300" "$CHANGEFEED_ID" 60 + + cdc_cli_changefeed pause --changefeed-id="$CHANGEFEED_ID" + sleep 2 + cdc_cli_changefeed update --config="$CUR/conf/changefeed.toml" --changefeed-id="$CHANGEFEED_ID" + cdc_cli_changefeed resume --changefeed-id="$CHANGEFEED_ID" + # Start node2 for moving the table. run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" From bc7a9cdd6817dbbb6b00a37eb9bb005c00015dc0 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 17 Dec 2025 16:42:49 +0800 Subject: [PATCH 03/20] update --- tests/integration_tests/syncpoint_move_table/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh index 879caea581..1f67581464 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -67,7 +67,7 @@ run() { cdc_cli_changefeed pause --changefeed-id="$CHANGEFEED_ID" sleep 2 - cdc_cli_changefeed update --config="$CUR/conf/changefeed.toml" --changefeed-id="$CHANGEFEED_ID" + cdc_cli_changefeed update --config="$CUR/conf/changefeed.toml" --changefeed-id="$CHANGEFEED_ID" --no-confirm cdc_cli_changefeed resume --changefeed-id="$CHANGEFEED_ID" # Start node2 for moving the table. From 83bb3009e0916486599ce47f1ba7b44bfa3b51c1 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 17 Dec 2025 22:33:07 +0800 Subject: [PATCH 04/20] update --- tests/integration_tests/run_heavy_it_in_ci.sh | 2 +- tests/integration_tests/syncpoint_move_table/run.sh | 7 ++----- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index f0f5d57e8f..72555e6859 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -38,7 +38,7 @@ mysql_groups=( # G03 'cdc move_table checkpoint_race_ddl_crash' # G04 - 'complex_transaction syncpoint syncpoint_check_ts random_drop_message' + 'complex_transaction syncpoint syncpoint_move_table syncpoint_check_ts random_drop_message' # G05 'ddl_for_split_tables_with_merge_and_split' # G06 diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh index 1f67581464..c59b00c9a0 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -35,9 +35,6 @@ run() { start_tidb_cluster --workdir $WORK_DIR - # This test contains `set global tidb_external_ts = ?`, which requires super privilege. - run_sql "SET GLOBAL tidb_enable_external_ts_read = on;" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" @@ -103,8 +100,8 @@ run() { fi # Wait until the syncpoint is written downstream, then validate snapshot consistency by sync_diff. - ensure 30 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" - secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) + ensure 30 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" + secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) echo "secondary_ts: $secondary_ts" deployConfig "$syncpoint_ts" "$secondary_ts" From 7d83951db3cad3c9ed9b5d0e032f3f3d90d44da3 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 18 Dec 2025 15:29:53 +0800 Subject: [PATCH 05/20] update --- .../dispatcher/basic_dispatcher.go | 6 +- .../dispatcher/redo_dispatcher.go | 5 +- .../dispatcher/redo_dispatcher_test.go | 1 + .../dispatchermanager/dispatcher_manager.go | 15 +- .../dispatcher_manager_helper.go | 6 +- .../dispatcher_manager_info.go | 5 + .../dispatcher_manager_redo.go | 10 +- .../dispatcher_manager_test.go | 51 +++ downstreamadapter/dispatchermanager/helper.go | 9 +- heartbeatpb/heartbeat.pb.go | 308 ++++++++++-------- heartbeatpb/heartbeat.proto | 5 + maintainer/replica/replication_span.go | 32 +- maintainer/replica/replication_span_test.go | 6 +- tests/integration_tests/ddl_move_table/run.sh | 149 +++++++++ tests/integration_tests/run_heavy_it_in_ci.sh | 2 +- .../syncpoint_move_table/run.sh | 35 +- 16 files changed, 474 insertions(+), 171 deletions(-) create mode 100644 tests/integration_tests/ddl_move_table/run.sh diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 61783a4b39..87c5094c8b 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -443,10 +443,10 @@ func (d *BasicDispatcher) handleEvents(dispatcherEvents []DispatcherEvent, wakeC } // Skip DML events at startTs+1 when skipDMLAsStartTs is true. - // This handles the corner case where a DDL at ts=X crashed after writing DML but before marking finished. - // We return startTs=X-1 to replay the DDL, but need to skip the already-written DML at ts=X (startTs+1). + // This flag is used when a dispatcher starts from (blockTs-1) to replay the DDL at blockTs, + // while avoiding potential duplicate DML writes at blockTs. if d.skipDMLAsStartTs && event.GetCommitTs() == d.startTs+1 { - log.Info("skip DML event at startTs+1 due to DDL crash recovery", + log.Info("skip DML event at startTs+1 due to skipDMLAsStartTs", zap.Stringer("dispatcher", d.id), zap.Uint64("startTs", d.startTs), zap.Uint64("dmlCommitTs", event.GetCommitTs()), diff --git a/downstreamadapter/dispatcher/redo_dispatcher.go b/downstreamadapter/dispatcher/redo_dispatcher.go index 1c14d97002..f43201ccc0 100644 --- a/downstreamadapter/dispatcher/redo_dispatcher.go +++ b/downstreamadapter/dispatcher/redo_dispatcher.go @@ -43,6 +43,7 @@ func NewRedoDispatcher( schemaID int64, schemaIDToDispatchers *SchemaIDToDispatchers, skipSyncpointAtStartTs bool, + skipDMLAsStartTs bool, sink sink.Sink, sharedInfo *SharedInfo, ) *RedoDispatcher { @@ -53,7 +54,9 @@ func NewRedoDispatcher( schemaID, schemaIDToDispatchers, skipSyncpointAtStartTs, - false, // skipDMLAsStartTs is not needed for redo dispatcher + // skipDMLAsStartTs is used when a dispatcher is recreated during an in-flight DDL barrier: + // we may start from (blockTs-1) to replay the DDL at blockTs, while skipping DML at blockTs (startTs+1). + skipDMLAsStartTs, 0, common.RedoMode, sink, diff --git a/downstreamadapter/dispatcher/redo_dispatcher_test.go b/downstreamadapter/dispatcher/redo_dispatcher_test.go index 01ab43e3a8..6ba642f7cb 100644 --- a/downstreamadapter/dispatcher/redo_dispatcher_test.go +++ b/downstreamadapter/dispatcher/redo_dispatcher_test.go @@ -57,6 +57,7 @@ func newRedoDispatcherForTest(sink sink.Sink, tableSpan *heartbeatpb.TableSpan) 1, // schemaID NewSchemaIDToDispatchers(), false, // skipSyncpointAtStartTs + false, // skipDMLAsStartTs sink, sharedInfo, ) diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager.go b/downstreamadapter/dispatchermanager/dispatcher_manager.go index 6e61200361..138e401613 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager.go @@ -383,7 +383,7 @@ func (e *DispatcherManager) newEventDispatchers(infos map[common.DispatcherID]di start := time.Now() currentPdTs := e.pdClock.CurrentTS() - dispatcherIds, tableIds, startTsList, tableSpans, schemaIds := prepareCreateDispatcher(infos, e.dispatcherMap) + dispatcherIds, tableIds, startTsList, tableSpans, schemaIds, scheduleSkipDMLAsStartTsList := prepareCreateDispatcher(infos, e.dispatcherMap) if len(dispatcherIds) == 0 { return nil } @@ -419,6 +419,14 @@ func (e *DispatcherManager) newEventDispatchers(infos map[common.DispatcherID]di } for idx, id := range dispatcherIds { + // if the newStartTs equals to the original startTs, we need to combine the skipDMLAsStartTs flag + // otherwise, we just use the skipDMLAsStartTs flag from mysql sink + var skipDMLAsStartTs bool + if newStartTsList[idx] == startTsList[idx] { + skipDMLAsStartTs = scheduleSkipDMLAsStartTsList[idx] || skipDMLAsStartTsList[idx] + } else { + skipDMLAsStartTs = skipDMLAsStartTsList[idx] + } d := dispatcher.NewEventDispatcher( id, tableSpans[idx], @@ -426,7 +434,7 @@ func (e *DispatcherManager) newEventDispatchers(infos map[common.DispatcherID]di schemaIds[idx], e.schemaIDToDispatchers, skipSyncpointAtStartTsList[idx], - skipDMLAsStartTsList[idx], + skipDMLAsStartTs, currentPdTs, e.sink, e.sharedInfo, @@ -463,7 +471,8 @@ func (e *DispatcherManager) newEventDispatchers(infos map[common.DispatcherID]di zap.Stringer("changefeedID", e.changefeedID), zap.Stringer("dispatcherID", id), zap.String("tableSpan", common.FormatTableSpan(tableSpans[idx])), - zap.Int64("startTs", newStartTsList[idx])) + zap.Int64("startTs", newStartTsList[idx]), + zap.Bool("skipDMLAsStartTs", skipDMLAsStartTs)) } e.metricCreateDispatcherDuration.Observe(time.Since(start).Seconds() / float64(len(dispatcherIds))) log.Info("batch create new dispatchers", diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go b/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go index 5f1a449905..70f8f75411 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go @@ -72,13 +72,14 @@ func getDispatcherStatus(id common.DispatcherID, dispatcherItem dispatcher.Dispa } func prepareCreateDispatcher[T dispatcher.Dispatcher](infos map[common.DispatcherID]dispatcherCreateInfo, dispatcherMap *DispatcherMap[T]) ( - []common.DispatcherID, []int64, []int64, []*heartbeatpb.TableSpan, []int64, + []common.DispatcherID, []int64, []int64, []*heartbeatpb.TableSpan, []int64, []bool, ) { dispatcherIds := make([]common.DispatcherID, 0, len(infos)) tableIds := make([]int64, 0, len(infos)) startTsList := make([]int64, 0, len(infos)) tableSpans := make([]*heartbeatpb.TableSpan, 0, len(infos)) schemaIds := make([]int64, 0, len(infos)) + skipDMLAsStartTsList := make([]bool, 0, len(infos)) for _, info := range infos { id := info.Id if _, ok := dispatcherMap.Get(id); ok { @@ -89,8 +90,9 @@ func prepareCreateDispatcher[T dispatcher.Dispatcher](infos map[common.Dispatche startTsList = append(startTsList, int64(info.StartTs)) tableSpans = append(tableSpans, info.TableSpan) schemaIds = append(schemaIds, info.SchemaID) + skipDMLAsStartTsList = append(skipDMLAsStartTsList, info.SkipDMLAsStartTs) } - return dispatcherIds, tableIds, startTsList, tableSpans, schemaIds + return dispatcherIds, tableIds, startTsList, tableSpans, schemaIds, skipDMLAsStartTsList } func prepareMergeDispatcher[T dispatcher.Dispatcher](changefeedID common.ChangeFeedID, diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_info.go b/downstreamadapter/dispatchermanager/dispatcher_manager_info.go index 59323164cb..2990883193 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_info.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_info.go @@ -27,6 +27,11 @@ type dispatcherCreateInfo struct { TableSpan *heartbeatpb.TableSpan StartTs uint64 SchemaID int64 + // SkipDMLAsStartTs indicates whether to skip DML events at (StartTs+1). + // It is used when a dispatcher is recreated during an in-flight DDL barrier: + // we need to replay the DDL by starting from (blockTs-1), while avoiding + // potential duplicate DML writes at blockTs. + SkipDMLAsStartTs bool } type cleanMap struct { diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go index 0ba4d234ea..f3b509a8ac 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go @@ -109,7 +109,7 @@ func (e *DispatcherManager) NewRedoTableTriggerEventDispatcher(id *heartbeatpb.D func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dispatcherCreateInfo, removeDDLTs bool) error { start := time.Now() - dispatcherIds, tableIds, startTsList, tableSpans, schemaIds := prepareCreateDispatcher(infos, e.redoDispatcherMap) + dispatcherIds, tableIds, startTsList, tableSpans, schemaIds, scheduleSkipDMLAsStartTsList := prepareCreateDispatcher(infos, e.redoDispatcherMap) if len(dispatcherIds) == 0 { return nil } @@ -122,12 +122,13 @@ func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dis // we will encounter a checkpoint-ts greater than the resolved-ts in the redo metadata. // This results in the redo metadata recording an incorrect log, which can cause a panic if no additional redo metadata logs are flushed. // Therefore, we must ensure that the start-ts remains consistent with the common dispatcher by querying the recovery info from the MySQL sink. - newStartTsList, _, _, err := e.getTableRecoveryInfoFromMysqlSink(tableIds, startTsList, removeDDLTs) + newStartTsList, _, skipDMLAsStartTsList, err := e.getTableRecoveryInfoFromMysqlSink(tableIds, startTsList, removeDDLTs) if err != nil { return errors.Trace(err) } for idx, id := range dispatcherIds { + skipDMLAsStartTs := skipDMLAsStartTsList[idx] || scheduleSkipDMLAsStartTsList[idx] rd := dispatcher.NewRedoDispatcher( id, tableSpans[idx], @@ -135,6 +136,7 @@ func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dis schemaIds[idx], e.redoSchemaIDToDispatchers, false, // skipSyncpointAtStartTs + skipDMLAsStartTs, e.redoSink, e.sharedInfo, ) @@ -162,7 +164,8 @@ func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dis zap.Stringer("changefeedID", e.changefeedID), zap.Stringer("dispatcherID", id), zap.String("tableSpan", common.FormatTableSpan(tableSpans[idx])), - zap.Int64("startTs", newStartTsList[idx])) + zap.Int64("startTs", newStartTsList[idx]), + zap.Bool("skipDMLAsStartTs", skipDMLAsStartTs)) } e.metricRedoCreateDispatcherDuration.Observe(time.Since(start).Seconds() / float64(len(dispatcherIds))) log.Info("batch create new redo dispatchers", @@ -197,6 +200,7 @@ func (e *DispatcherManager) mergeRedoDispatcher(dispatcherIDs []common.Dispatche schemaID, e.redoSchemaIDToDispatchers, false, // skipSyncpointAtStartTs + false, // skipDMLAsStartTs e.redoSink, e.sharedInfo, ) diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_test.go b/downstreamadapter/dispatchermanager/dispatcher_manager_test.go index 61684288f5..3cdda91e6d 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_test.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_test.go @@ -125,6 +125,57 @@ func createTestManager(t *testing.T) *DispatcherManager { return manager } +func TestNewEventDispatchers_PropagateSkipDMLAsStartTs(t *testing.T) { + manager := createTestManager(t) + manager.metricTableTriggerEventDispatcherCount = metrics.TableTriggerEventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "tableTrigger") + manager.metricCreateDispatcherDuration = metrics.CreateDispatcherDuration.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "eventDispatcher") + + dispatcherID := common.NewDispatcherID() + totalSpan := common.TableIDToComparableSpan(0, 1) + infos := map[common.DispatcherID]dispatcherCreateInfo{ + dispatcherID: { + Id: dispatcherID, + TableSpan: &heartbeatpb.TableSpan{TableID: totalSpan.TableID, StartKey: totalSpan.StartKey, EndKey: totalSpan.EndKey}, + StartTs: 99, + SchemaID: 1, + + SkipDMLAsStartTs: true, + }, + } + require.NoError(t, manager.newEventDispatchers(infos, false)) + + d, ok := manager.dispatcherMap.Get(dispatcherID) + require.True(t, ok) + require.True(t, d.GetSkipDMLAsStartTs()) +} + +func TestNewRedoDispatchers_PropagateSkipDMLAsStartTs(t *testing.T) { + manager := createTestManager(t) + manager.redoDispatcherMap = newDispatcherMap[*dispatcher.RedoDispatcher]() + manager.redoSchemaIDToDispatchers = dispatcher.NewSchemaIDToDispatchers() + manager.redoSink = nil + manager.metricRedoTableTriggerEventDispatcherCount = metrics.TableTriggerEventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") + manager.metricRedoEventDispatcherCount = metrics.EventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") + manager.metricRedoCreateDispatcherDuration = metrics.CreateDispatcherDuration.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") + + dispatcherID := common.NewDispatcherID() + infos := map[common.DispatcherID]dispatcherCreateInfo{ + dispatcherID: { + Id: dispatcherID, + TableSpan: common.KeyspaceDDLSpan(0), + StartTs: 99, + SchemaID: 0, + + SkipDMLAsStartTs: true, + }, + } + require.NoError(t, manager.newRedoDispatchers(infos, false)) + + d, ok := manager.redoDispatcherMap.Get(dispatcherID) + require.True(t, ok) + require.True(t, d.GetSkipDMLAsStartTs()) +} + func TestMergeDispatcherNormal(t *testing.T) { manager := createTestManager(t) diff --git a/downstreamadapter/dispatchermanager/helper.go b/downstreamadapter/dispatchermanager/helper.go index da4333c3e6..17cf4eed4c 100644 --- a/downstreamadapter/dispatchermanager/helper.go +++ b/downstreamadapter/dispatchermanager/helper.go @@ -201,10 +201,11 @@ func (h *SchedulerDispatcherRequestHandler) Handle(dispatcherManager *Dispatcher switch req.ScheduleAction { case heartbeatpb.ScheduleAction_Create: info := dispatcherCreateInfo{ - Id: dispatcherID, - TableSpan: config.Span, - StartTs: config.StartTs, - SchemaID: config.SchemaID, + Id: dispatcherID, + TableSpan: config.Span, + StartTs: config.StartTs, + SchemaID: config.SchemaID, + SkipDMLAsStartTs: config.SkipDMLAsStartTs, } if common.IsRedoMode(config.Mode) { redoInfos[dispatcherID] = info diff --git a/heartbeatpb/heartbeat.pb.go b/heartbeatpb/heartbeat.pb.go index 321405d004..d0005657af 100644 --- a/heartbeatpb/heartbeat.pb.go +++ b/heartbeatpb/heartbeat.pb.go @@ -813,6 +813,11 @@ type DispatcherConfig struct { DispatcherID *DispatcherID `protobuf:"bytes,3,opt,name=dispatcherID,proto3" json:"dispatcherID,omitempty"` SchemaID int64 `protobuf:"varint,4,opt,name=schemaID,proto3" json:"schemaID,omitempty"` Mode int64 `protobuf:"varint,5,opt,name=mode,proto3" json:"mode,omitempty"` + // skipDMLAsStartTs indicates whether to skip DML events at (startTs+1). + // It is mainly used when a dispatcher is recreated during an in-flight DDL barrier: + // we need to replay the DDL at blockTs by starting from (blockTs-1), while avoiding + // potential duplicate DML writes at blockTs. + SkipDMLAsStartTs bool `protobuf:"varint,6,opt,name=skipDMLAsStartTs,proto3" json:"skipDMLAsStartTs,omitempty"` } func (m *DispatcherConfig) Reset() { *m = DispatcherConfig{} } @@ -883,6 +888,13 @@ func (m *DispatcherConfig) GetMode() int64 { return 0 } +func (m *DispatcherConfig) GetSkipDMLAsStartTs() bool { + if m != nil { + return m.SkipDMLAsStartTs + } + return false +} + type ScheduleDispatcherRequest struct { ChangefeedID *ChangefeedID `protobuf:"bytes,1,opt,name=changefeedID,proto3" json:"changefeedID,omitempty"` Config *DispatcherConfig `protobuf:"bytes,2,opt,name=config,proto3" json:"config,omitempty"` @@ -2789,138 +2801,139 @@ func init() { func init() { proto.RegisterFile("heartbeatpb/heartbeat.proto", fileDescriptor_6d584080fdadb670) } var fileDescriptor_6d584080fdadb670 = []byte{ - // 2084 bytes of a gzipped FileDescriptorProto + // 2104 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x19, 0x4d, 0x6f, 0x1c, 0x49, - 0xd5, 0xdd, 0x3d, 0x9f, 0x6f, 0x3c, 0x76, 0xa7, 0xb2, 0x71, 0x26, 0xb1, 0xe3, 0x38, 0xcd, 0x87, - 0x8c, 0x17, 0x1c, 0xc5, 0xbb, 0x11, 0x1f, 0x5a, 0x08, 0xf6, 0x4c, 0xd8, 0x1d, 0x99, 0x78, 0xad, - 0x1a, 0xa3, 0xf0, 0x71, 0x18, 0xda, 0xdd, 0x95, 0x71, 0xcb, 0x33, 0xdd, 0x9d, 0xae, 0x9e, 0x38, - 0x8e, 0x84, 0x10, 0x42, 0xdc, 0x38, 0xec, 0x95, 0x23, 0x7f, 0x00, 0x71, 0xe1, 0x88, 0xc4, 0x0d, - 0x24, 0x2e, 0x7b, 0x5a, 0x71, 0x44, 0x89, 0xb8, 0x23, 0x2e, 0x5c, 0x51, 0x55, 0x57, 0xf5, 0xd7, - 0xf4, 0x78, 0x6c, 0x79, 0xc4, 0xad, 0x5f, 0xd5, 0x7b, 0xaf, 0x5e, 0xbd, 0xef, 0x7a, 0x0d, 0xab, - 0x27, 0xc4, 0x0c, 0xc2, 0x63, 0x62, 0x86, 0xfe, 0xf1, 0xc3, 0xf8, 0x7b, 0xdb, 0x0f, 0xbc, 0xd0, - 0x43, 0x8d, 0xd4, 0xa6, 0x71, 0x0e, 0xf5, 0x23, 0xf3, 0x78, 0x48, 0x7a, 0xbe, 0xe9, 0xa2, 0x16, - 0x54, 0x39, 0xd0, 0xed, 0xb4, 0x94, 0x0d, 0x65, 0x53, 0xc3, 0x12, 0x44, 0x77, 0xa1, 0xd6, 0x0b, - 0xcd, 0x20, 0xdc, 0x27, 0xe7, 0x2d, 0x75, 0x43, 0xd9, 0x5c, 0xc4, 0x31, 0x8c, 0x56, 0xa0, 0xf2, - 0xd4, 0xb5, 0xd9, 0x8e, 0xc6, 0x77, 0x04, 0x84, 0xd6, 0x01, 0xf6, 0xc9, 0x39, 0xf5, 0x4d, 0x8b, - 0x31, 0x2c, 0x6d, 0x28, 0x9b, 0x4d, 0x9c, 0x5a, 0x31, 0xbe, 0x50, 0x41, 0xff, 0x84, 0x89, 0xb2, - 0x47, 0xcc, 0x10, 0x93, 0x97, 0x63, 0x42, 0x43, 0xf4, 0x5d, 0x58, 0xb4, 0x4e, 0x4c, 0x77, 0x40, - 0x5e, 0x10, 0x62, 0x0b, 0x39, 0x1a, 0x3b, 0x77, 0xb6, 0x53, 0x32, 0x6f, 0xb7, 0x53, 0x08, 0x38, - 0x83, 0x8e, 0x3e, 0x84, 0xfa, 0x99, 0x19, 0x92, 0x60, 0x64, 0x06, 0xa7, 0x5c, 0xd0, 0xc6, 0xce, - 0x4a, 0x86, 0xf6, 0xb9, 0xdc, 0xc5, 0x09, 0x22, 0xfa, 0x08, 0x9a, 0x01, 0xb1, 0xbd, 0x78, 0x8f, - 0x5f, 0x64, 0x3a, 0x65, 0x16, 0x19, 0x7d, 0x0b, 0x6a, 0x34, 0x34, 0xc3, 0x31, 0x25, 0xb4, 0x55, - 0xda, 0xd0, 0x36, 0x1b, 0x3b, 0x6b, 0x19, 0xc2, 0x58, 0xbf, 0x3d, 0x8e, 0x85, 0x63, 0x6c, 0xb4, - 0x09, 0xcb, 0x96, 0x37, 0xf2, 0xc9, 0x90, 0x84, 0x24, 0xda, 0x6c, 0x95, 0x37, 0x94, 0xcd, 0x1a, - 0xce, 0x2f, 0xa3, 0xf7, 0x41, 0x23, 0x41, 0xd0, 0xaa, 0x14, 0x68, 0x03, 0x8f, 0x5d, 0xd7, 0x71, - 0x07, 0x4f, 0x83, 0xc0, 0x0b, 0x30, 0xc3, 0x32, 0x7e, 0xa3, 0x40, 0x3d, 0x11, 0xcf, 0x60, 0x1a, - 0x25, 0xd6, 0xa9, 0xef, 0x39, 0x6e, 0x78, 0x44, 0xb9, 0x46, 0x4b, 0x38, 0xb3, 0xc6, 0x4c, 0x15, - 0x10, 0xea, 0x0d, 0x5f, 0x11, 0xfb, 0x88, 0x72, 0xbd, 0x95, 0x70, 0x6a, 0x05, 0xe9, 0xa0, 0x51, - 0xf2, 0x92, 0xab, 0xa5, 0x84, 0xd9, 0x27, 0xe3, 0x3a, 0x34, 0x69, 0xd8, 0x3b, 0x77, 0x2d, 0x4e, - 0x53, 0x8a, 0xb8, 0xa6, 0xd7, 0x8c, 0x5f, 0x80, 0xde, 0x71, 0xa8, 0x6f, 0x86, 0xd6, 0x09, 0x09, - 0x76, 0xad, 0xd0, 0xf1, 0x5c, 0xf4, 0x3e, 0x54, 0x4c, 0xfe, 0xc5, 0xe5, 0x58, 0xda, 0xb9, 0x99, - 0xb9, 0x4b, 0x84, 0x84, 0x05, 0x0a, 0xf3, 0xba, 0xb6, 0x37, 0x1a, 0x39, 0x61, 0x2c, 0x54, 0x0c, - 0xa3, 0x0d, 0x68, 0x74, 0x29, 0x3b, 0xea, 0x90, 0xdd, 0x81, 0x8b, 0x56, 0xc3, 0xe9, 0x25, 0xa3, - 0x0d, 0xda, 0x6e, 0x7b, 0x3f, 0xc3, 0x44, 0xb9, 0x98, 0x89, 0x3a, 0xc9, 0xe4, 0xd7, 0x2a, 0xdc, - 0xea, 0xba, 0x2f, 0x86, 0x63, 0xc2, 0x2e, 0x95, 0x5c, 0x87, 0xa2, 0xef, 0x43, 0x33, 0xde, 0x38, - 0x3a, 0xf7, 0x89, 0xb8, 0xd0, 0xdd, 0xcc, 0x85, 0x32, 0x18, 0x38, 0x4b, 0x80, 0x9e, 0x40, 0x33, - 0x61, 0xd8, 0xed, 0xb0, 0x3b, 0x6a, 0x13, 0xe6, 0x4d, 0x63, 0xe0, 0x2c, 0x3e, 0x8f, 0x4a, 0xeb, - 0x84, 0x8c, 0xcc, 0x6e, 0x87, 0x2b, 0x40, 0xc3, 0x31, 0x8c, 0xf6, 0xe1, 0x26, 0x79, 0x6d, 0x0d, - 0xc7, 0x36, 0x49, 0xd1, 0xd8, 0xdc, 0x4e, 0x17, 0x1e, 0x51, 0x44, 0x65, 0xfc, 0x55, 0x49, 0x9b, - 0x52, 0xf8, 0xe4, 0x8f, 0xe1, 0x96, 0x53, 0xa4, 0x19, 0x11, 0xb3, 0x46, 0xb1, 0x22, 0xd2, 0x98, - 0xb8, 0x98, 0x01, 0x7a, 0x1c, 0x3b, 0x49, 0x14, 0xc2, 0xf7, 0xa6, 0x88, 0x9b, 0x73, 0x17, 0x03, - 0x34, 0xd3, 0x92, 0xc1, 0xab, 0x67, 0x1d, 0xab, 0xbd, 0x8f, 0xd9, 0xa6, 0xf1, 0x27, 0x05, 0x6e, - 0xa4, 0x92, 0x0e, 0xf5, 0x3d, 0x97, 0x92, 0xeb, 0x66, 0x9d, 0x67, 0x80, 0xec, 0x9c, 0x76, 0x88, - 0xb4, 0xe6, 0x34, 0xd9, 0x45, 0x32, 0x28, 0x20, 0x44, 0x08, 0x4a, 0x23, 0xcf, 0x26, 0xc2, 0xa4, - 0xfc, 0xdb, 0x78, 0x0d, 0x37, 0xdb, 0xa9, 0x88, 0x7d, 0x46, 0x28, 0x35, 0x07, 0xd7, 0x16, 0x3c, - 0x9f, 0x1b, 0xd4, 0xc9, 0xdc, 0x60, 0x7c, 0xa6, 0x40, 0x03, 0x13, 0xdb, 0x9b, 0xd3, 0x91, 0xb3, - 0x52, 0x4d, 0x5e, 0x24, 0xad, 0x40, 0xa4, 0xac, 0x3b, 0xb6, 0x3d, 0xf7, 0x85, 0x33, 0x40, 0x5b, - 0x50, 0xa2, 0xbe, 0xe9, 0x0a, 0x79, 0x56, 0x8a, 0x53, 0x30, 0xe6, 0x38, 0xac, 0xd0, 0x51, 0x56, - 0xbe, 0x62, 0x09, 0x24, 0xc8, 0x6e, 0x67, 0xa7, 0xc2, 0x41, 0x38, 0xd3, 0x05, 0xf1, 0x92, 0x41, - 0x67, 0x11, 0x49, 0x65, 0x44, 0x96, 0xa2, 0x88, 0x94, 0x70, 0x6c, 0xd6, 0x72, 0xca, 0xac, 0x5f, - 0x28, 0x70, 0x87, 0x85, 0xac, 0x3d, 0x1e, 0xa6, 0x22, 0x6e, 0x4e, 0xc5, 0xf0, 0x31, 0x54, 0x2c, - 0xae, 0x9b, 0x19, 0x61, 0x14, 0x29, 0x10, 0x0b, 0x64, 0xd4, 0x86, 0x25, 0x2a, 0x44, 0x8a, 0x02, - 0x8c, 0x2b, 0x61, 0x69, 0x67, 0x35, 0x43, 0xde, 0xcb, 0xa0, 0xe0, 0x1c, 0x89, 0xf1, 0x5f, 0x05, - 0x56, 0x9e, 0x91, 0x60, 0x30, 0xff, 0x5b, 0x3d, 0x81, 0xa6, 0x7d, 0xc5, 0xac, 0x99, 0xc1, 0x47, - 0x5d, 0x40, 0x23, 0x26, 0x99, 0xdd, 0xb9, 0x92, 0xa1, 0x0b, 0x88, 0x62, 0x93, 0x96, 0x52, 0x26, - 0x3d, 0x84, 0x9b, 0xcf, 0x4c, 0xc7, 0x0d, 0x4d, 0xc7, 0x25, 0xc1, 0x27, 0x92, 0x1b, 0xfa, 0x76, - 0xaa, 0x4b, 0x50, 0x0a, 0x32, 0x43, 0x42, 0x93, 0x6f, 0x13, 0x8c, 0xbf, 0xa8, 0xa0, 0xe7, 0xb7, - 0xaf, 0xab, 0xc5, 0x7b, 0x00, 0xec, 0xab, 0xcf, 0x0e, 0x21, 0xdc, 0x3f, 0xea, 0xb8, 0xce, 0x56, - 0x18, 0x7b, 0x82, 0x1e, 0x41, 0x39, 0xda, 0x29, 0x32, 0x7d, 0xdb, 0x1b, 0xf9, 0x9e, 0x4b, 0xdc, - 0x90, 0xe3, 0xe2, 0x08, 0x13, 0x7d, 0x09, 0x9a, 0x49, 0x90, 0xf6, 0xc3, 0xb8, 0x25, 0xc8, 0x34, - 0x1a, 0xa2, 0x8f, 0x29, 0x17, 0x98, 0x6c, 0xa2, 0x8f, 0x41, 0x5f, 0x81, 0xa5, 0x63, 0xcf, 0x0b, - 0x69, 0x18, 0x98, 0x7e, 0xdf, 0xf6, 0x5c, 0xc2, 0xfb, 0x9f, 0x1a, 0x6e, 0xc6, 0xab, 0x1d, 0xcf, - 0x25, 0x13, 0xad, 0x48, 0xb5, 0xa0, 0x15, 0xf9, 0x26, 0xac, 0xb6, 0x3d, 0x2f, 0xb0, 0x1d, 0xd7, - 0x0c, 0xbd, 0x60, 0x4f, 0xd2, 0x4b, 0x97, 0x6c, 0x41, 0xf5, 0x15, 0x09, 0xa8, 0x6c, 0x4b, 0x34, - 0x2c, 0x41, 0xe3, 0x27, 0xb0, 0x56, 0x4c, 0x28, 0x2a, 0xc7, 0x35, 0xcc, 0xfa, 0x77, 0x05, 0xde, - 0xdb, 0xb5, 0xed, 0x04, 0x43, 0x4a, 0xf3, 0x35, 0x50, 0x1d, 0x7b, 0xb6, 0x41, 0x55, 0xc7, 0x66, - 0xbd, 0x77, 0x2a, 0xc4, 0x17, 0xe3, 0x18, 0x9e, 0x30, 0x46, 0x41, 0x1a, 0x45, 0x5b, 0x70, 0xc3, - 0xa1, 0x7d, 0x97, 0x9c, 0xf5, 0x13, 0xd7, 0xe0, 0x56, 0xab, 0xe1, 0x65, 0x87, 0x1e, 0x90, 0xb3, - 0xe4, 0x38, 0x74, 0x1f, 0x1a, 0xa7, 0xa2, 0x75, 0xef, 0x3b, 0x36, 0xcf, 0x61, 0x4d, 0x0c, 0x72, - 0xa9, 0x6b, 0x1b, 0xbf, 0x53, 0xe0, 0x36, 0x26, 0x23, 0xef, 0x15, 0xb9, 0xd6, 0x85, 0x5a, 0x50, - 0xb5, 0x4c, 0x6a, 0x99, 0x36, 0x11, 0xdd, 0x98, 0x04, 0xd9, 0x4e, 0xc0, 0xf9, 0xdb, 0xa2, 0xd9, - 0x93, 0x60, 0x5e, 0xb6, 0xd2, 0x84, 0x6c, 0x7f, 0xd0, 0xe0, 0x6e, 0x22, 0xd5, 0x84, 0xf5, 0xaf, - 0x19, 0x4a, 0xd3, 0x6c, 0x70, 0x87, 0xbb, 0x46, 0x90, 0x52, 0x7f, 0x5c, 0x65, 0x2c, 0x78, 0x10, - 0xb2, 0x92, 0xd4, 0x0f, 0x03, 0x67, 0x30, 0x20, 0x41, 0x9f, 0xbc, 0x22, 0x6e, 0xd8, 0x4f, 0xf2, - 0x94, 0xbc, 0xc7, 0x85, 0x19, 0xe9, 0x1e, 0xe7, 0x71, 0x14, 0xb1, 0x78, 0xca, 0x38, 0xa4, 0x9b, - 0xb6, 0x62, 0xf3, 0x96, 0x8b, 0xcd, 0x3b, 0x84, 0xaf, 0xb2, 0x47, 0x4d, 0x7f, 0xb6, 0x54, 0x95, - 0x59, 0x52, 0x3d, 0x60, 0x8c, 0x8e, 0x2e, 0x94, 0x2c, 0x67, 0xb0, 0xea, 0x84, 0xc1, 0xfe, 0xa5, - 0xc0, 0x6a, 0xa1, 0xc1, 0xe6, 0xd3, 0xaf, 0x3d, 0x86, 0x32, 0x6b, 0x03, 0x64, 0xe9, 0xb8, 0x9f, - 0xa1, 0x8b, 0x4f, 0x4b, 0x9a, 0x86, 0x08, 0x5b, 0x26, 0x2f, 0xed, 0x32, 0x8f, 0xb0, 0x4b, 0xa5, - 0x43, 0x56, 0x25, 0xd7, 0x93, 0x7b, 0x1e, 0x7a, 0x34, 0x9c, 0xb7, 0x73, 0x5e, 0xca, 0xd3, 0xd4, - 0x6b, 0x7a, 0xda, 0x23, 0xa8, 0x46, 0x5d, 0x0e, 0x73, 0x74, 0xa6, 0xd1, 0xdb, 0x13, 0xad, 0xc2, - 0xc8, 0xec, 0xba, 0x2f, 0x3c, 0x2c, 0xf1, 0x8c, 0xff, 0x28, 0x70, 0x7f, 0xea, 0xcd, 0xe7, 0x63, - 0xe5, 0xff, 0xcb, 0xd5, 0xaf, 0xe2, 0x13, 0xc6, 0x6b, 0x80, 0x44, 0x17, 0x99, 0xd7, 0x9b, 0x92, - 0x7b, 0xbd, 0xad, 0x4b, 0xcc, 0x03, 0x73, 0x24, 0xcb, 0x73, 0x6a, 0x05, 0x6d, 0x43, 0x85, 0xbb, - 0xa7, 0x54, 0x78, 0x41, 0xbb, 0xcb, 0xf5, 0x2d, 0xb0, 0x8c, 0xb6, 0x18, 0xf3, 0xf0, 0x83, 0xa7, - 0x8f, 0x79, 0xd6, 0x04, 0x5a, 0xea, 0xd4, 0x64, 0xc1, 0xf8, 0xb3, 0x0a, 0x68, 0x32, 0x3a, 0x58, - 0x76, 0x9f, 0x62, 0x9c, 0x8c, 0x22, 0x55, 0x31, 0x46, 0x92, 0x57, 0x56, 0x73, 0x57, 0x96, 0xfd, - 0xbb, 0x76, 0x89, 0xfe, 0xfd, 0x07, 0xa0, 0x5b, 0xb2, 0x09, 0xe9, 0x47, 0x05, 0x95, 0xc7, 0xd7, - 0x8c, 0x4e, 0x65, 0xd9, 0x4a, 0xc3, 0x63, 0x3a, 0x19, 0xa4, 0xe5, 0x82, 0x32, 0xf9, 0x01, 0x34, - 0x8e, 0x87, 0x9e, 0x75, 0x2a, 0x7a, 0xa5, 0x28, 0x01, 0xa2, 0xac, 0x87, 0x73, 0xf6, 0xc0, 0xd1, - 0xa2, 0x06, 0x4a, 0x76, 0x86, 0xd5, 0x54, 0x67, 0xf8, 0x12, 0x56, 0x12, 0x97, 0x6f, 0x0f, 0x3d, - 0x4a, 0xe6, 0x14, 0xe4, 0xa9, 0xd2, 0xa8, 0x66, 0x4a, 0xa3, 0x11, 0xc0, 0xed, 0x89, 0x23, 0xe7, - 0x13, 0x5d, 0xec, 0x09, 0x35, 0xb6, 0x2c, 0x42, 0xa9, 0x3c, 0x53, 0x80, 0xc6, 0x6f, 0x15, 0xd0, - 0x93, 0xe7, 0x7e, 0xe4, 0x80, 0x73, 0x98, 0x96, 0xdc, 0x85, 0x9a, 0x70, 0xd3, 0x28, 0x6f, 0x6b, - 0x38, 0x86, 0x2f, 0x1a, 0x84, 0x18, 0x3f, 0x83, 0x32, 0xc7, 0x9b, 0x31, 0xdd, 0x9c, 0xe6, 0x96, - 0x6b, 0x50, 0xef, 0xf9, 0x43, 0x87, 0x67, 0x01, 0xd1, 0x78, 0x24, 0x0b, 0x86, 0x0b, 0x4b, 0x12, - 0x33, 0xd2, 0xd5, 0x05, 0xa7, 0x6c, 0x40, 0xe3, 0xd3, 0xa1, 0x9d, 0x3b, 0x28, 0xbd, 0xc4, 0x30, - 0x0e, 0xc8, 0x59, 0xee, 0x26, 0xe9, 0x25, 0xe3, 0xf7, 0x1a, 0x94, 0x23, 0x07, 0x5b, 0x83, 0x7a, - 0x97, 0xee, 0x31, 0x87, 0x23, 0x51, 0x6b, 0x55, 0xc3, 0xc9, 0x02, 0x93, 0x82, 0x7f, 0x26, 0x0f, - 0x5c, 0x01, 0xa2, 0x27, 0xd0, 0x88, 0x3e, 0x65, 0xfa, 0x98, 0x7c, 0x19, 0xe6, 0x8d, 0x87, 0xd3, - 0x14, 0x68, 0x1f, 0x6e, 0x1c, 0x10, 0x62, 0x77, 0x02, 0xcf, 0xf7, 0x25, 0x86, 0xe8, 0x55, 0x66, - 0xb0, 0x99, 0xa4, 0x43, 0x1f, 0xc1, 0x32, 0x5b, 0xdc, 0xb5, 0xed, 0x98, 0x55, 0xf4, 0x36, 0x40, - 0x93, 0xf1, 0x8f, 0xf3, 0xa8, 0xec, 0xa5, 0xfa, 0x23, 0xdf, 0x36, 0x43, 0x22, 0x54, 0x48, 0x5b, - 0x15, 0x4e, 0xbc, 0x5a, 0x54, 0x7e, 0x84, 0x81, 0x70, 0x8e, 0x24, 0x3f, 0x03, 0xac, 0x4e, 0xcc, - 0x00, 0xd1, 0x37, 0xf8, 0x63, 0x68, 0x40, 0x5a, 0x35, 0xee, 0xb3, 0xd9, 0xe2, 0xb6, 0x27, 0x62, - 0x7e, 0x10, 0x3d, 0x84, 0x06, 0xc4, 0xf8, 0x25, 0xbc, 0x17, 0xe7, 0x2b, 0xb9, 0xcb, 0x92, 0xcd, - 0x15, 0xf2, 0xe4, 0xa6, 0x7c, 0x7e, 0xa9, 0x53, 0x93, 0x8d, 0x78, 0x75, 0x15, 0xcd, 0x8a, 0xfe, - 0xad, 0xc0, 0x72, 0x6e, 0xe8, 0x7c, 0x95, 0xc3, 0x8b, 0x92, 0xab, 0x3a, 0x8f, 0xe4, 0x5a, 0xf4, - 0x06, 0x79, 0x04, 0xb7, 0xa2, 0xb2, 0x4c, 0x9d, 0x37, 0xa4, 0xef, 0x93, 0xa0, 0x4f, 0x89, 0xe5, - 0xb9, 0x51, 0xf7, 0xab, 0x62, 0xc4, 0x37, 0x7b, 0xce, 0x1b, 0x72, 0x48, 0x82, 0x1e, 0xdf, 0x29, - 0x9c, 0xa3, 0xfc, 0x51, 0x01, 0x94, 0xd2, 0xf5, 0x9c, 0xf2, 0xea, 0xc7, 0xd0, 0x3c, 0x4e, 0x98, - 0xc6, 0x23, 0xbd, 0x07, 0xc5, 0xb5, 0x29, 0x7d, 0x7e, 0x96, 0xae, 0xd0, 0x4a, 0x36, 0x2c, 0xa6, - 0x3b, 0x04, 0x86, 0x13, 0x3a, 0xa3, 0x28, 0x31, 0xd6, 0x31, 0xff, 0x66, 0x6b, 0xae, 0x67, 0xcb, - 0x52, 0xcc, 0xbf, 0xd9, 0x9a, 0x25, 0x79, 0xd5, 0x31, 0xff, 0x66, 0xe1, 0x3e, 0x8a, 0xc6, 0x73, - 0x5c, 0x6f, 0x75, 0x2c, 0x41, 0xe3, 0x43, 0x58, 0xcc, 0x4f, 0x2c, 0x4e, 0x9c, 0xc1, 0x89, 0x98, - 0x84, 0xf3, 0x6f, 0xa4, 0x83, 0x36, 0xf4, 0xce, 0x44, 0xa2, 0x60, 0x9f, 0x4c, 0xb6, 0xb4, 0x5a, - 0x2e, 0x47, 0xc5, 0xa5, 0x65, 0x8d, 0x83, 0x90, 0x8c, 0x7d, 0xb3, 0xd4, 0x2a, 0xfb, 0x7a, 0x21, - 0x5a, 0x0c, 0x1b, 0x3f, 0x87, 0xfb, 0x3f, 0xf4, 0x06, 0xa9, 0xe7, 0x35, 0x8e, 0xc7, 0x80, 0xf3, - 0x31, 0xa0, 0xf1, 0x2b, 0x05, 0x36, 0xa6, 0x1f, 0x31, 0x9f, 0x42, 0x38, 0x63, 0xa0, 0xb9, 0x75, - 0x0f, 0x2a, 0xe2, 0xdf, 0x47, 0x1d, 0xca, 0xcf, 0x03, 0x27, 0x24, 0xfa, 0x02, 0xaa, 0x41, 0xe9, - 0xd0, 0xa4, 0x54, 0x57, 0xb6, 0x36, 0xa3, 0x0a, 0x92, 0x8c, 0xce, 0x10, 0x40, 0xa5, 0x1d, 0x10, - 0x93, 0xe3, 0x01, 0x54, 0xa2, 0x47, 0xb5, 0xae, 0x6c, 0x7d, 0x07, 0x20, 0x49, 0x36, 0x8c, 0xc3, - 0xc1, 0xa7, 0x07, 0x4f, 0xf5, 0x05, 0xd4, 0x80, 0xea, 0xf3, 0xdd, 0xee, 0x51, 0xf7, 0xe0, 0x63, - 0x5d, 0xe1, 0x00, 0x8e, 0x00, 0x95, 0xe1, 0x74, 0x18, 0x8e, 0xb6, 0xf5, 0xf5, 0x5c, 0xf9, 0x45, - 0x55, 0xd0, 0x76, 0x87, 0x43, 0x7d, 0x01, 0x55, 0x40, 0xed, 0xec, 0xe9, 0x0a, 0x3b, 0xe9, 0xc0, - 0x0b, 0x46, 0xe6, 0x50, 0x57, 0xb7, 0xde, 0xc0, 0x52, 0x36, 0xb8, 0x39, 0x5b, 0x2f, 0x38, 0x75, - 0xdc, 0x41, 0x74, 0x60, 0x2f, 0xe4, 0x59, 0x3c, 0x3a, 0x30, 0x92, 0xd0, 0xd6, 0x55, 0xa4, 0xc3, - 0x62, 0xd7, 0x75, 0x42, 0xc7, 0x1c, 0x3a, 0x6f, 0x18, 0xae, 0x86, 0x9a, 0x50, 0x3f, 0x0c, 0x88, - 0x6f, 0x06, 0x0c, 0x2c, 0xa1, 0x25, 0x00, 0x3e, 0x15, 0xc4, 0xc4, 0xb4, 0xcf, 0xf5, 0x32, 0x23, - 0x78, 0x6e, 0x3a, 0xa1, 0xe3, 0x0e, 0xf8, 0xb2, 0x5e, 0xd9, 0xfb, 0xde, 0xdf, 0xde, 0xae, 0x2b, - 0x9f, 0xbf, 0x5d, 0x57, 0xfe, 0xf9, 0x76, 0x5d, 0xf9, 0xec, 0xdd, 0xfa, 0xc2, 0xe7, 0xef, 0xd6, - 0x17, 0xfe, 0xf1, 0x6e, 0x7d, 0xe1, 0xa7, 0x5f, 0x1e, 0x38, 0xe1, 0xc9, 0xf8, 0x78, 0xdb, 0xf2, - 0x46, 0x0f, 0x7d, 0xc7, 0x1d, 0x58, 0xa6, 0xff, 0x30, 0x74, 0x2c, 0xdb, 0x7a, 0x98, 0xb2, 0xd4, - 0x71, 0x85, 0xff, 0xe4, 0xfc, 0xe0, 0x7f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd2, 0x79, 0xcd, 0x26, - 0x03, 0x1d, 0x00, 0x00, + 0xd5, 0xdd, 0x3d, 0x1f, 0x9e, 0x37, 0x1e, 0xbb, 0x53, 0xd9, 0x38, 0x4e, 0xec, 0x38, 0x4e, 0xf3, + 0x21, 0xe3, 0x05, 0x47, 0xf1, 0x6e, 0xc4, 0x87, 0x16, 0x82, 0x3d, 0x13, 0x76, 0x47, 0x5e, 0x7b, + 0xad, 0x1a, 0xa3, 0xf0, 0x71, 0x18, 0xda, 0xdd, 0x95, 0x71, 0xcb, 0x33, 0xdd, 0x9d, 0xae, 0x76, + 0x1c, 0x47, 0x42, 0x08, 0x21, 0x6e, 0x1c, 0xf6, 0xca, 0x91, 0x3f, 0x80, 0xb8, 0x70, 0x44, 0xe2, + 0x06, 0x12, 0x97, 0x3d, 0x45, 0x1c, 0x51, 0x22, 0xee, 0x08, 0x0e, 0x5c, 0x51, 0x55, 0x57, 0xf5, + 0xf7, 0x78, 0x6c, 0x79, 0xc4, 0xad, 0xeb, 0xd5, 0x7b, 0xaf, 0x5e, 0xbd, 0xef, 0x7a, 0x0d, 0xcb, + 0xc7, 0xc4, 0x0c, 0xc2, 0x23, 0x62, 0x86, 0xfe, 0xd1, 0xc3, 0xf8, 0x7b, 0xd3, 0x0f, 0xbc, 0xd0, + 0x43, 0xcd, 0xd4, 0xa6, 0x71, 0x0e, 0x8d, 0x43, 0xf3, 0x68, 0x48, 0x7a, 0xbe, 0xe9, 0xa2, 0x25, + 0xa8, 0xf3, 0x45, 0xb7, 0xb3, 0xa4, 0xac, 0x29, 0xeb, 0x1a, 0x96, 0x4b, 0x74, 0x17, 0x66, 0x7b, + 0xa1, 0x19, 0x84, 0xbb, 0xe4, 0x7c, 0x49, 0x5d, 0x53, 0xd6, 0xe7, 0x70, 0xbc, 0x46, 0x8b, 0x50, + 0x7b, 0xea, 0xda, 0x6c, 0x47, 0xe3, 0x3b, 0x62, 0x85, 0x56, 0x01, 0x76, 0xc9, 0x39, 0xf5, 0x4d, + 0x8b, 0x31, 0xac, 0xac, 0x29, 0xeb, 0x2d, 0x9c, 0x82, 0x18, 0x6f, 0x54, 0xd0, 0x3f, 0x61, 0xa2, + 0xec, 0x10, 0x33, 0xc4, 0xe4, 0xc5, 0x29, 0xa1, 0x21, 0xfa, 0x2e, 0xcc, 0x59, 0xc7, 0xa6, 0x3b, + 0x20, 0xcf, 0x09, 0xb1, 0x85, 0x1c, 0xcd, 0xad, 0x3b, 0x9b, 0x29, 0x99, 0x37, 0xdb, 0x29, 0x04, + 0x9c, 0x41, 0x47, 0x1f, 0x42, 0xe3, 0xcc, 0x0c, 0x49, 0x30, 0x32, 0x83, 0x13, 0x2e, 0x68, 0x73, + 0x6b, 0x31, 0x43, 0xfb, 0x4c, 0xee, 0xe2, 0x04, 0x11, 0x7d, 0x04, 0xad, 0x80, 0xd8, 0x5e, 0xbc, + 0xc7, 0x2f, 0x32, 0x9e, 0x32, 0x8b, 0x8c, 0xbe, 0x05, 0xb3, 0x34, 0x34, 0xc3, 0x53, 0x4a, 0xe8, + 0x52, 0x65, 0x4d, 0x5b, 0x6f, 0x6e, 0xad, 0x64, 0x08, 0x63, 0xfd, 0xf6, 0x38, 0x16, 0x8e, 0xb1, + 0xd1, 0x3a, 0x2c, 0x58, 0xde, 0xc8, 0x27, 0x43, 0x12, 0x92, 0x68, 0x73, 0xa9, 0xba, 0xa6, 0xac, + 0xcf, 0xe2, 0x3c, 0x18, 0xbd, 0x0f, 0x1a, 0x09, 0x82, 0xa5, 0x5a, 0x89, 0x36, 0xf0, 0xa9, 0xeb, + 0x3a, 0xee, 0xe0, 0x69, 0x10, 0x78, 0x01, 0x66, 0x58, 0xc6, 0xaf, 0x15, 0x68, 0x24, 0xe2, 0x19, + 0x4c, 0xa3, 0xc4, 0x3a, 0xf1, 0x3d, 0xc7, 0x0d, 0x0f, 0x29, 0xd7, 0x68, 0x05, 0x67, 0x60, 0xcc, + 0x54, 0x01, 0xa1, 0xde, 0xf0, 0x25, 0xb1, 0x0f, 0x29, 0xd7, 0x5b, 0x05, 0xa7, 0x20, 0x48, 0x07, + 0x8d, 0x92, 0x17, 0x5c, 0x2d, 0x15, 0xcc, 0x3e, 0x19, 0xd7, 0xa1, 0x49, 0xc3, 0xde, 0xb9, 0x6b, + 0x71, 0x9a, 0x4a, 0xc4, 0x35, 0x0d, 0x33, 0x7e, 0x0e, 0x7a, 0xc7, 0xa1, 0xbe, 0x19, 0x5a, 0xc7, + 0x24, 0xd8, 0xb6, 0x42, 0xc7, 0x73, 0xd1, 0xfb, 0x50, 0x33, 0xf9, 0x17, 0x97, 0x63, 0x7e, 0xeb, + 0x66, 0xe6, 0x2e, 0x11, 0x12, 0x16, 0x28, 0xcc, 0xeb, 0xda, 0xde, 0x68, 0xe4, 0x84, 0xb1, 0x50, + 0xf1, 0x1a, 0xad, 0x41, 0xb3, 0x4b, 0xd9, 0x51, 0x07, 0xec, 0x0e, 0x5c, 0xb4, 0x59, 0x9c, 0x06, + 0x19, 0x6d, 0xd0, 0xb6, 0xdb, 0xbb, 0x19, 0x26, 0xca, 0xc5, 0x4c, 0xd4, 0x22, 0x93, 0x5f, 0xa9, + 0x70, 0xab, 0xeb, 0x3e, 0x1f, 0x9e, 0x12, 0x76, 0xa9, 0xe4, 0x3a, 0x14, 0x7d, 0x1f, 0x5a, 0xf1, + 0xc6, 0xe1, 0xb9, 0x4f, 0xc4, 0x85, 0xee, 0x66, 0x2e, 0x94, 0xc1, 0xc0, 0x59, 0x02, 0xf4, 0x04, + 0x5a, 0x09, 0xc3, 0x6e, 0x87, 0xdd, 0x51, 0x2b, 0x98, 0x37, 0x8d, 0x81, 0xb3, 0xf8, 0x3c, 0x2a, + 0xad, 0x63, 0x32, 0x32, 0xbb, 0x1d, 0xae, 0x00, 0x0d, 0xc7, 0x6b, 0xb4, 0x0b, 0x37, 0xc9, 0x2b, + 0x6b, 0x78, 0x6a, 0x93, 0x14, 0x8d, 0xcd, 0xed, 0x74, 0xe1, 0x11, 0x65, 0x54, 0xc6, 0x5f, 0x94, + 0xb4, 0x29, 0x85, 0x4f, 0xfe, 0x08, 0x6e, 0x39, 0x65, 0x9a, 0x11, 0x31, 0x6b, 0x94, 0x2b, 0x22, + 0x8d, 0x89, 0xcb, 0x19, 0xa0, 0xc7, 0xb1, 0x93, 0x44, 0x21, 0x7c, 0x6f, 0x8c, 0xb8, 0x39, 0x77, + 0x31, 0x40, 0x33, 0x2d, 0x19, 0xbc, 0x7a, 0xd6, 0xb1, 0xda, 0xbb, 0x98, 0x6d, 0x1a, 0x7f, 0x54, + 0xe0, 0x46, 0x2a, 0xe9, 0x50, 0xdf, 0x73, 0x29, 0xb9, 0x6e, 0xd6, 0xd9, 0x03, 0x64, 0xe7, 0xb4, + 0x43, 0xa4, 0x35, 0xc7, 0xc9, 0x2e, 0x92, 0x41, 0x09, 0x21, 0x42, 0x50, 0x19, 0x79, 0x36, 0x11, + 0x26, 0xe5, 0xdf, 0xc6, 0x2b, 0xb8, 0xd9, 0x4e, 0x45, 0xec, 0x1e, 0xa1, 0xd4, 0x1c, 0x5c, 0x5b, + 0xf0, 0x7c, 0x6e, 0x50, 0x8b, 0xb9, 0xc1, 0xf8, 0x5c, 0x81, 0x26, 0x26, 0xb6, 0x37, 0xa5, 0x23, + 0x27, 0xa5, 0x9a, 0xbc, 0x48, 0x5a, 0x89, 0x48, 0xff, 0xc9, 0xb8, 0x63, 0xdb, 0x73, 0x9f, 0x3b, + 0x03, 0xb4, 0x01, 0x15, 0xea, 0x9b, 0xae, 0x90, 0x67, 0xb1, 0x3c, 0x05, 0x63, 0x8e, 0xc3, 0x0a, + 0x1d, 0x65, 0xe5, 0x2b, 0x96, 0x40, 0x2e, 0xd9, 0xed, 0xec, 0x54, 0x38, 0x08, 0x67, 0xba, 0x20, + 0x5e, 0x32, 0xe8, 0x2c, 0x22, 0xa9, 0x8c, 0xc8, 0x4a, 0x14, 0x91, 0x72, 0x1d, 0x9b, 0xb5, 0x9a, + 0x98, 0x15, 0x6d, 0x80, 0x4e, 0x4f, 0x1c, 0xbf, 0xb3, 0xf7, 0xe9, 0x36, 0xed, 0x09, 0x89, 0x6a, + 0x3c, 0x0b, 0x15, 0xe0, 0xc6, 0x1b, 0x05, 0xee, 0xb0, 0xf0, 0xb6, 0x4f, 0x87, 0xa9, 0xe8, 0x9c, + 0x52, 0xe1, 0x7c, 0x0c, 0x35, 0x8b, 0xeb, 0x71, 0x42, 0xc8, 0x45, 0xca, 0xc6, 0x02, 0x19, 0xb5, + 0x61, 0x9e, 0x0a, 0x91, 0xa2, 0x60, 0xe4, 0x0a, 0x9b, 0xdf, 0x5a, 0xce, 0x90, 0xf7, 0x32, 0x28, + 0x38, 0x47, 0x62, 0xfc, 0x57, 0x81, 0xc5, 0x3d, 0x12, 0x0c, 0xa6, 0x7f, 0xab, 0x27, 0xd0, 0xb2, + 0xaf, 0x98, 0x61, 0x33, 0xf8, 0xa8, 0x0b, 0x68, 0xc4, 0x24, 0xb3, 0x3b, 0x57, 0x72, 0x8a, 0x12, + 0xa2, 0xd8, 0xfc, 0x95, 0x54, 0x54, 0x1f, 0xc0, 0xcd, 0x3d, 0xd3, 0x71, 0x43, 0xd3, 0x71, 0x49, + 0xf0, 0x89, 0xe4, 0x86, 0xbe, 0x9d, 0xea, 0x28, 0x94, 0x92, 0x2c, 0x92, 0xd0, 0xe4, 0x5b, 0x0a, + 0xe3, 0xcf, 0x2a, 0xe8, 0xf9, 0xed, 0xeb, 0x6a, 0xf1, 0x1e, 0x00, 0xfb, 0xea, 0xb3, 0x43, 0x08, + 0xf7, 0x8f, 0x06, 0x6e, 0x30, 0x08, 0x63, 0x4f, 0xd0, 0x23, 0xa8, 0x46, 0x3b, 0x65, 0xa6, 0x6f, + 0x7b, 0x23, 0xdf, 0x73, 0x89, 0x1b, 0x72, 0x5c, 0x1c, 0x61, 0xa2, 0x2f, 0x41, 0x2b, 0x09, 0xe8, + 0x7e, 0x18, 0xb7, 0x0f, 0x99, 0xa6, 0x44, 0xf4, 0x3c, 0xd5, 0x12, 0x93, 0x15, 0x7a, 0x1e, 0xf4, + 0x15, 0x98, 0x3f, 0xf2, 0xbc, 0x90, 0x86, 0x81, 0xe9, 0xf7, 0x6d, 0xcf, 0x25, 0x22, 0x8c, 0x5a, + 0x31, 0xb4, 0xe3, 0xb9, 0xa4, 0xd0, 0xb6, 0xd4, 0x4b, 0xda, 0x96, 0x6f, 0xc2, 0x72, 0xdb, 0xf3, + 0x02, 0xdb, 0x71, 0xcd, 0xd0, 0x0b, 0x76, 0x24, 0xbd, 0x74, 0xc9, 0x25, 0xa8, 0xbf, 0x24, 0x01, + 0x95, 0x2d, 0x8c, 0x86, 0xe5, 0xd2, 0xf8, 0x31, 0xac, 0x94, 0x13, 0x8a, 0x2a, 0x73, 0x0d, 0xb3, + 0xfe, 0x4d, 0x81, 0xf7, 0xb6, 0x6d, 0x3b, 0xc1, 0x90, 0xd2, 0x7c, 0x0d, 0x54, 0xc7, 0x9e, 0x6c, + 0x50, 0xd5, 0xb1, 0x59, 0x9f, 0x9e, 0x0a, 0xf1, 0xb9, 0x38, 0x86, 0x0b, 0xc6, 0x28, 0x49, 0xb9, + 0x68, 0x03, 0x6e, 0x38, 0xb4, 0xef, 0x92, 0xb3, 0x7e, 0xe2, 0x1a, 0xdc, 0x6a, 0xb3, 0x78, 0xc1, + 0xa1, 0xfb, 0xe4, 0x2c, 0x39, 0x0e, 0xdd, 0x87, 0xe6, 0x89, 0x68, 0xf3, 0xfb, 0x8e, 0xcd, 0xf3, + 0x5d, 0x0b, 0x83, 0x04, 0x75, 0x6d, 0xe3, 0xb7, 0x0a, 0xdc, 0xc6, 0x64, 0xe4, 0xbd, 0x24, 0xd7, + 0xba, 0xd0, 0x12, 0xd4, 0x2d, 0x93, 0x5a, 0xa6, 0x4d, 0x44, 0xe7, 0x26, 0x97, 0x6c, 0x27, 0xe0, + 0xfc, 0x6d, 0xd1, 0x18, 0xca, 0x65, 0x5e, 0xb6, 0x4a, 0x41, 0xb6, 0xdf, 0x6b, 0x70, 0x37, 0x91, + 0xaa, 0x60, 0xfd, 0x6b, 0x86, 0xd2, 0x38, 0x1b, 0xdc, 0xe1, 0xae, 0x11, 0xa4, 0xd4, 0x1f, 0x57, + 0x24, 0x0b, 0x1e, 0x84, 0xac, 0x7c, 0xf5, 0xc3, 0xc0, 0x19, 0x0c, 0x48, 0xd0, 0x27, 0x2f, 0x89, + 0x1b, 0xf6, 0x93, 0x3c, 0x25, 0xef, 0x71, 0x61, 0x46, 0xba, 0xc7, 0x79, 0x1c, 0x46, 0x2c, 0x9e, + 0x32, 0x0e, 0xe9, 0x06, 0xaf, 0xdc, 0xbc, 0xd5, 0x72, 0xf3, 0x0e, 0xe1, 0xab, 0xec, 0x01, 0xd4, + 0x9f, 0x2c, 0x55, 0x6d, 0x92, 0x54, 0x0f, 0x18, 0xa3, 0xc3, 0x0b, 0x25, 0xcb, 0x19, 0xac, 0x5e, + 0x30, 0xd8, 0x3f, 0x15, 0x58, 0x2e, 0x35, 0xd8, 0x74, 0x7a, 0xbb, 0xc7, 0x50, 0x65, 0x2d, 0x83, + 0x2c, 0x1d, 0xf7, 0x33, 0x74, 0xf1, 0x69, 0x49, 0x83, 0x11, 0x61, 0xcb, 0xe4, 0xa5, 0x5d, 0xe6, + 0xc1, 0x76, 0xa9, 0x74, 0xc8, 0xaa, 0xe4, 0x6a, 0x72, 0xcf, 0x03, 0x8f, 0x86, 0xd3, 0x76, 0xce, + 0x4b, 0x79, 0x9a, 0x7a, 0x4d, 0x4f, 0x7b, 0x04, 0xf5, 0xa8, 0x23, 0x62, 0x8e, 0xce, 0x34, 0x7a, + 0xbb, 0xd0, 0x2a, 0x8c, 0xcc, 0xae, 0xfb, 0xdc, 0xc3, 0x12, 0xcf, 0xf8, 0xb7, 0x02, 0xf7, 0xc7, + 0xde, 0x7c, 0x3a, 0x56, 0xfe, 0xbf, 0x5c, 0xfd, 0x2a, 0x3e, 0x61, 0xbc, 0x02, 0x48, 0x74, 0x91, + 0x79, 0xe9, 0x29, 0xb9, 0x97, 0xde, 0xaa, 0xc4, 0xdc, 0x37, 0x47, 0xb2, 0x3c, 0xa7, 0x20, 0x68, + 0x13, 0x6a, 0xdc, 0x3d, 0xa5, 0xc2, 0x4b, 0x5a, 0x63, 0xae, 0x6f, 0x81, 0x65, 0xb4, 0xc5, 0x48, + 0x88, 0x1f, 0x3c, 0x7e, 0x24, 0xb4, 0x22, 0xd0, 0x52, 0xa7, 0x26, 0x00, 0xe3, 0x4f, 0x2a, 0xa0, + 0x62, 0x74, 0xb0, 0xec, 0x3e, 0xc6, 0x38, 0x19, 0x45, 0xaa, 0x62, 0xe4, 0x24, 0xaf, 0xac, 0xe6, + 0xae, 0x2c, 0x7b, 0x7d, 0xed, 0x12, 0xbd, 0xfe, 0x0f, 0x40, 0xb7, 0x64, 0x13, 0xd2, 0x8f, 0x0a, + 0x2a, 0x8f, 0xaf, 0x09, 0x9d, 0xca, 0x82, 0x95, 0x5e, 0x9f, 0xd2, 0x62, 0x90, 0x56, 0x4b, 0xca, + 0xe4, 0x07, 0xd0, 0x3c, 0x1a, 0x7a, 0xd6, 0x89, 0xe8, 0x95, 0xa2, 0x04, 0x88, 0xb2, 0x1e, 0xce, + 0xd9, 0x03, 0x47, 0x8b, 0x1a, 0x28, 0xd9, 0x19, 0xd6, 0x53, 0x9d, 0xe1, 0x0b, 0x58, 0x4c, 0x5c, + 0xbe, 0x3d, 0xf4, 0x28, 0x99, 0x52, 0x90, 0xa7, 0x4a, 0xa3, 0x9a, 0x29, 0x8d, 0x46, 0x00, 0xb7, + 0x0b, 0x47, 0x4e, 0x27, 0xba, 0xd8, 0x73, 0xeb, 0xd4, 0xb2, 0x08, 0xa5, 0xf2, 0x4c, 0xb1, 0x34, + 0x7e, 0xa3, 0x80, 0x9e, 0x8c, 0x06, 0x22, 0x07, 0x9c, 0xc2, 0x64, 0xe5, 0x2e, 0xcc, 0x0a, 0x37, + 0x8d, 0xf2, 0xb6, 0x86, 0xe3, 0xf5, 0x45, 0x43, 0x13, 0xe3, 0xa7, 0x50, 0xe5, 0x78, 0x13, 0x26, + 0xa1, 0xe3, 0xdc, 0x72, 0x05, 0x1a, 0x3d, 0x7f, 0xe8, 0xf0, 0x2c, 0x20, 0x1a, 0x8f, 0x04, 0x60, + 0xb8, 0x30, 0x2f, 0x31, 0x23, 0x5d, 0x5d, 0x70, 0xca, 0x1a, 0x34, 0x3f, 0x1b, 0xda, 0xb9, 0x83, + 0xd2, 0x20, 0x86, 0xb1, 0x4f, 0xce, 0x72, 0x37, 0x49, 0x83, 0x8c, 0xdf, 0x69, 0x50, 0x8d, 0x1c, + 0x6c, 0x05, 0x1a, 0x5d, 0xba, 0xc3, 0x1c, 0x8e, 0x44, 0xad, 0xd5, 0x2c, 0x4e, 0x00, 0x4c, 0x0a, + 0xfe, 0x99, 0x3c, 0x86, 0xc5, 0x12, 0x3d, 0x81, 0x66, 0xf4, 0x29, 0xd3, 0x47, 0xf1, 0x65, 0x98, + 0x37, 0x1e, 0x4e, 0x53, 0xa0, 0x5d, 0xb8, 0xb1, 0x4f, 0x88, 0xdd, 0x09, 0x3c, 0xdf, 0x97, 0x18, + 0xa2, 0x57, 0x99, 0xc0, 0xa6, 0x48, 0x87, 0x3e, 0x82, 0x05, 0x06, 0xdc, 0xb6, 0xed, 0x98, 0x55, + 0xf4, 0x36, 0x40, 0xc5, 0xf8, 0xc7, 0x79, 0x54, 0xf6, 0x52, 0xfd, 0xa1, 0x6f, 0x9b, 0x21, 0x11, + 0x2a, 0x64, 0xef, 0x6c, 0x46, 0xbc, 0x5c, 0x56, 0x7e, 0x84, 0x81, 0x70, 0x8e, 0x24, 0x3f, 0x2f, + 0xac, 0x17, 0xe6, 0x85, 0xe8, 0x1b, 0xfc, 0x31, 0x34, 0x20, 0x4b, 0xb3, 0xdc, 0x67, 0xb3, 0xc5, + 0x6d, 0x47, 0xc4, 0xfc, 0x20, 0x7a, 0x08, 0x0d, 0x88, 0xf1, 0x0b, 0x78, 0x2f, 0xce, 0x57, 0x72, + 0x97, 0x25, 0x9b, 0x2b, 0xe4, 0xc9, 0x75, 0xf9, 0xfc, 0x52, 0xc7, 0x26, 0x1b, 0xf1, 0xea, 0x2a, + 0x9b, 0x2b, 0xfd, 0x4b, 0x81, 0x85, 0xdc, 0x80, 0xfa, 0x2a, 0x87, 0x97, 0x25, 0x57, 0x75, 0x1a, + 0xc9, 0xb5, 0xec, 0x0d, 0xf2, 0x08, 0x6e, 0x45, 0x65, 0x99, 0x3a, 0xaf, 0x49, 0xdf, 0x27, 0x41, + 0x9f, 0x12, 0xcb, 0x73, 0xa3, 0xee, 0x57, 0xc5, 0x88, 0x6f, 0xf6, 0x9c, 0xd7, 0xe4, 0x80, 0x04, + 0x3d, 0xbe, 0x53, 0x36, 0x73, 0x31, 0xfe, 0xa0, 0x00, 0x4a, 0xe9, 0x7a, 0x4a, 0x79, 0xf5, 0x63, + 0x68, 0x1d, 0x25, 0x4c, 0xe3, 0xf1, 0xdf, 0x83, 0xf2, 0xda, 0x94, 0x3e, 0x3f, 0x4b, 0x57, 0x6a, + 0x25, 0x1b, 0xe6, 0xd2, 0x1d, 0x02, 0xc3, 0x09, 0x9d, 0x51, 0x94, 0x18, 0x1b, 0x98, 0x7f, 0x33, + 0x98, 0xeb, 0xd9, 0xb2, 0x14, 0xf3, 0x6f, 0x06, 0xb3, 0x24, 0xaf, 0x06, 0xe6, 0xdf, 0x2c, 0xdc, + 0x47, 0xd1, 0x28, 0x8f, 0xeb, 0xad, 0x81, 0xe5, 0xd2, 0xf8, 0x10, 0xe6, 0xf2, 0x13, 0x8b, 0x63, + 0x67, 0x70, 0x2c, 0xa6, 0xe6, 0xfc, 0x1b, 0xe9, 0xa0, 0x0d, 0xbd, 0x33, 0x91, 0x28, 0xd8, 0x27, + 0x93, 0x2d, 0xad, 0x96, 0xcb, 0x51, 0x71, 0x69, 0x59, 0xe3, 0x20, 0x24, 0x63, 0xdf, 0x2c, 0xb5, + 0xca, 0xbe, 0x5e, 0x88, 0x16, 0xaf, 0x8d, 0x9f, 0xc1, 0xfd, 0x4f, 0xbd, 0x41, 0xea, 0x79, 0x8d, + 0xe3, 0x91, 0xe1, 0x74, 0x0c, 0x68, 0xfc, 0x52, 0x81, 0xb5, 0xf1, 0x47, 0x4c, 0xa7, 0x10, 0x4e, + 0x18, 0x7e, 0x6e, 0xdc, 0x83, 0x9a, 0xf8, 0x4f, 0xd2, 0x80, 0xea, 0xb3, 0xc0, 0x09, 0x89, 0x3e, + 0x83, 0x66, 0xa1, 0x72, 0x60, 0x52, 0xaa, 0x2b, 0x1b, 0xeb, 0x51, 0x05, 0x49, 0x46, 0x67, 0x08, + 0xa0, 0xd6, 0x0e, 0x88, 0xc9, 0xf1, 0x00, 0x6a, 0xd1, 0xa3, 0x5a, 0x57, 0x36, 0xbe, 0x03, 0x90, + 0x24, 0x1b, 0xc6, 0x61, 0xff, 0xb3, 0xfd, 0xa7, 0xfa, 0x0c, 0x6a, 0x42, 0xfd, 0xd9, 0x76, 0xf7, + 0xb0, 0xbb, 0xff, 0xb1, 0xae, 0xf0, 0x05, 0x8e, 0x16, 0x2a, 0xc3, 0xe9, 0x30, 0x1c, 0x6d, 0xe3, + 0xeb, 0xb9, 0xf2, 0x8b, 0xea, 0xa0, 0x6d, 0x0f, 0x87, 0xfa, 0x0c, 0xaa, 0x81, 0xda, 0xd9, 0xd1, + 0x15, 0x76, 0xd2, 0xbe, 0x17, 0x8c, 0xcc, 0xa1, 0xae, 0x6e, 0xbc, 0x86, 0xf9, 0x6c, 0x70, 0x73, + 0xb6, 0x5e, 0x70, 0xe2, 0xb8, 0x83, 0xe8, 0xc0, 0x5e, 0xc8, 0xb3, 0x78, 0x74, 0x60, 0x24, 0xa1, + 0xad, 0xab, 0x48, 0x87, 0xb9, 0xae, 0xeb, 0x84, 0x8e, 0x39, 0x74, 0x5e, 0x33, 0x5c, 0x0d, 0xb5, + 0xa0, 0x71, 0x10, 0x10, 0xdf, 0x0c, 0xd8, 0xb2, 0x82, 0xe6, 0x01, 0xf8, 0x54, 0x10, 0x13, 0xd3, + 0x3e, 0xd7, 0xab, 0x8c, 0xe0, 0x99, 0xe9, 0x84, 0x8e, 0x3b, 0xe0, 0x60, 0xbd, 0xb6, 0xf3, 0xbd, + 0xbf, 0xbe, 0x5d, 0x55, 0xbe, 0x78, 0xbb, 0xaa, 0xfc, 0xe3, 0xed, 0xaa, 0xf2, 0xf9, 0xbb, 0xd5, + 0x99, 0x2f, 0xde, 0xad, 0xce, 0xfc, 0xfd, 0xdd, 0xea, 0xcc, 0x4f, 0xbe, 0x3c, 0x70, 0xc2, 0xe3, + 0xd3, 0xa3, 0x4d, 0xcb, 0x1b, 0x3d, 0xf4, 0x1d, 0x77, 0x60, 0x99, 0xfe, 0xc3, 0xd0, 0xb1, 0x6c, + 0xeb, 0x61, 0xca, 0x52, 0x47, 0x35, 0xfe, 0x43, 0xf4, 0x83, 0xff, 0x05, 0x00, 0x00, 0xff, 0xff, + 0x9b, 0xbc, 0xac, 0xd4, 0x2f, 0x1d, 0x00, 0x00, } func (m *TableSpan) Marshal() (dAtA []byte, err error) { @@ -3466,6 +3479,16 @@ func (m *DispatcherConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.SkipDMLAsStartTs { + i-- + if m.SkipDMLAsStartTs { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x30 + } if m.Mode != 0 { i = encodeVarintHeartbeat(dAtA, i, uint64(m.Mode)) i-- @@ -5310,6 +5333,9 @@ func (m *DispatcherConfig) Size() (n int) { if m.Mode != 0 { n += 1 + sovHeartbeat(uint64(m.Mode)) } + if m.SkipDMLAsStartTs { + n += 2 + } return n } @@ -7524,6 +7550,26 @@ func (m *DispatcherConfig) Unmarshal(dAtA []byte) error { break } } + case 6: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field SkipDMLAsStartTs", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowHeartbeat + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.SkipDMLAsStartTs = bool(v != 0) default: iNdEx = preIndex skippy, err := skipHeartbeat(dAtA[iNdEx:]) diff --git a/heartbeatpb/heartbeat.proto b/heartbeatpb/heartbeat.proto index da0061acdb..4242f281b6 100644 --- a/heartbeatpb/heartbeat.proto +++ b/heartbeatpb/heartbeat.proto @@ -86,6 +86,11 @@ message DispatcherConfig { DispatcherID dispatcherID = 3; int64 schemaID = 4; int64 mode = 5; + // skipDMLAsStartTs indicates whether to skip DML events at (startTs+1). + // It is mainly used when a dispatcher is recreated during an in-flight DDL barrier: + // we need to replay the DDL at blockTs by starting from (blockTs-1), while avoiding + // potential duplicate DML writes at blockTs. + bool skipDMLAsStartTs = 6; } message ScheduleDispatcherRequest { diff --git a/maintainer/replica/replication_span.go b/maintainer/replica/replication_span.go index 1924616b9d..c03e8ee627 100644 --- a/maintainer/replica/replication_span.go +++ b/maintainer/replica/replication_span.go @@ -236,23 +236,39 @@ func (r *SpanReplication) NewAddDispatcherMessage(server node.ID) *messaging.Tar // dispatcher can only enter the syncpoint barrier after all events with commitTs <= BlockTs have been pushed // downstream. startTs := r.status.Load().CheckpointTs + skipDMLAsStartTs := false if state := r.blockState.Load(); state != nil && state.IsBlocked && - state.IsSyncPoint && (state.Stage == heartbeatpb.BlockStage_WAITING || state.Stage == heartbeatpb.BlockStage_WRITING) && - state.BlockTs > startTs { - startTs = state.BlockTs + state.BlockTs > 0 { + if state.IsSyncPoint { + if state.BlockTs > startTs { + startTs = state.BlockTs + } + } else { + // For an in-flight DDL barrier, a recreated dispatcher must start from (blockTs-1) so that it can + // replay the DDL at blockTs. At the same time, it should skip DML events at blockTs to avoid potential + // duplicate DML writes when the dispatcher is moved/recreated during the barrier. + blockTsMinusOne := state.BlockTs - 1 + if blockTsMinusOne > startTs { + startTs = blockTsMinusOne + } + if startTs+1 == state.BlockTs { + skipDMLAsStartTs = true + } + } } return messaging.NewSingleTargetMessage(server, messaging.HeartbeatCollectorTopic, &heartbeatpb.ScheduleDispatcherRequest{ ChangefeedID: r.ChangefeedID.ToPB(), Config: &heartbeatpb.DispatcherConfig{ - DispatcherID: r.ID.ToPB(), - SchemaID: r.schemaID, - Span: r.Span, - StartTs: startTs, - Mode: r.GetMode(), + DispatcherID: r.ID.ToPB(), + SchemaID: r.schemaID, + Span: r.Span, + StartTs: startTs, + SkipDMLAsStartTs: skipDMLAsStartTs, + Mode: r.GetMode(), }, ScheduleAction: heartbeatpb.ScheduleAction_Create, }) diff --git a/maintainer/replica/replication_span_test.go b/maintainer/replica/replication_span_test.go index 93bf973421..4c6253d9ff 100644 --- a/maintainer/replica/replication_span_test.go +++ b/maintainer/replica/replication_span_test.go @@ -54,6 +54,7 @@ func TestSpanReplication_NewAddDispatcherMessage(t *testing.T) { require.Equal(t, replicaSet.ID.ToPB(), req.Config.DispatcherID) require.Equal(t, replicaSet.schemaID, req.Config.SchemaID) require.Equal(t, uint64(10), req.Config.StartTs) + require.False(t, req.Config.SkipDMLAsStartTs) } func TestSpanReplication_NewAddDispatcherMessage_UseBlockTsForInFlightSyncPoint(t *testing.T) { @@ -70,6 +71,7 @@ func TestSpanReplication_NewAddDispatcherMessage_UseBlockTsForInFlightSyncPoint( msg := replicaSet.NewAddDispatcherMessage("node1") req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) require.Equal(t, uint64(10), req.Config.StartTs) + require.False(t, req.Config.SkipDMLAsStartTs) } func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsAfterSyncPointDone(t *testing.T) { @@ -86,9 +88,10 @@ func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsAfterSyncPointDon msg := replicaSet.NewAddDispatcherMessage("node1") req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) require.Equal(t, uint64(20), req.Config.StartTs) + require.False(t, req.Config.SkipDMLAsStartTs) } -func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsForDDL(t *testing.T) { +func TestSpanReplication_NewAddDispatcherMessage_UseBlockTsMinusOneForDDLInFlight(t *testing.T) { t.Parallel() replicaSet := NewSpanReplication(common.NewChangeFeedIDWithName("test", common.DefaultKeyspaceNamme), common.NewDispatcherID(), 1, getTableSpanByID(4), 9, common.DefaultMode, false) @@ -102,6 +105,7 @@ func TestSpanReplication_NewAddDispatcherMessage_DontUseBlockTsForDDL(t *testing msg := replicaSet.NewAddDispatcherMessage("node1") req := msg.Message[0].(*heartbeatpb.ScheduleDispatcherRequest) require.Equal(t, uint64(9), req.Config.StartTs) + require.True(t, req.Config.SkipDMLAsStartTs) } // getTableSpanByID returns a mock TableSpan for testing diff --git a/tests/integration_tests/ddl_move_table/run.sh b/tests/integration_tests/ddl_move_table/run.sh new file mode 100644 index 0000000000..5308447dbc --- /dev/null +++ b/tests/integration_tests/ddl_move_table/run.sh @@ -0,0 +1,149 @@ +#!/bin/bash +# +# This test verifies that moving a table dispatcher during an in-flight multi-table DDL barrier +# does not cause the moved dispatcher to miss the DDL, and that the recreated dispatcher starts +# from (blockTs-1) with skipDMLAsStartTs enabled. + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare + +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +DB_NAME="ddl_move" +TABLE_1="t1" +TABLE_2="t2" +TABLE_1_NEW="t1_new" +TABLE_2_NEW="t2_new" +CHANGEFEED_ID="test" + +deployDiffConfig() { + cat >$WORK_DIR/diff_config.toml <>>>>>" diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index 72555e6859..5bb3bb803c 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -36,7 +36,7 @@ mysql_groups=( # G02 'ddl_for_split_tables_with_failover' # G03 - 'cdc move_table checkpoint_race_ddl_crash' + 'cdc move_table ddl_move_table checkpoint_race_ddl_crash' # G04 'complex_transaction syncpoint syncpoint_move_table syncpoint_check_ts random_drop_message' # G05 diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh index c59b00c9a0..f5650ace42 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -17,9 +17,11 @@ SINK_TYPE=$1 DB_NAME="sp_move" TABLE_NAME="t" CHANGEFEED_ID="test" +dml_pid=0 deployConfig() { cat $CUR/conf/diff_config_part1.toml >$CUR/conf/diff_config.toml + sed -i "s#output-dir = \"/tmp/tidb_cdc_test/syncpoint_move_table/sync_diff/output\"#output-dir = \"$WORK_DIR/sync_diff/output\"#g" $CUR/conf/diff_config.toml echo "snapshot = \"$1\"" >>$CUR/conf/diff_config.toml cat $CUR/conf/diff_config_part2.toml >>$CUR/conf/diff_config.toml echo "snapshot = \"$2\"" >>$CUR/conf/diff_config.toml @@ -56,13 +58,12 @@ run() { kill_cdc_pid $cdc_pid_1 cleanup_process $CDC_BINARY - export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(90000)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(20000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-1" --addr "127.0.0.1:8300" check_coordinator_and_maintainer "127.0.0.1:8300" "$CHANGEFEED_ID" 60 - - cdc_cli_changefeed pause --changefeed-id="$CHANGEFEED_ID" + cdc_cli_changefeed pause --changefeed-id="$CHANGEFEED_ID" sleep 2 cdc_cli_changefeed update --config="$CUR/conf/changefeed.toml" --changefeed-id="$CHANGEFEED_ID" --no-confirm cdc_cli_changefeed resume --changefeed-id="$CHANGEFEED_ID" @@ -70,14 +71,17 @@ run() { # Start node2 for moving the table. run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" - # Keep generating DML to ensure a syncpoint is triggered. - for i in $(seq 10 50); do + # Keep generating DML until a syncpoint is triggered. + i=10 + while true; do run_sql "INSERT INTO ${DB_NAME}.${TABLE_NAME} (id, v) VALUES (${i}, ${i}) ON DUPLICATE KEY UPDATE v = v + 1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + i=$((i + 1)) sleep 1 - done + done & + dml_pid=$! # Wait for the table-trigger dispatcher to receive a syncpoint event and extract its commitTs. - ensure 30 "grep -q \"dispatcher receive sync point event\" $WORK_DIR/cdc0-1.log" + ensure 60 "grep -q \"dispatcher receive sync point event\" $WORK_DIR/cdc0-1.log" syncpoint_ts=$(grep "dispatcher receive sync point event" $WORK_DIR/cdc0-1.log | head -n 1 | grep -oE 'commitTs[^0-9]*[0-9]+' | head -n 1 | grep -oE '[0-9]+' || true) if [ -z "$syncpoint_ts" ]; then echo "failed to extract syncpoint commitTs from logs" @@ -86,33 +90,36 @@ run() { echo "syncpoint_ts: $syncpoint_ts" # Ensure the table-trigger dispatcher has received the maintainer action to start writing this syncpoint. - ensure 30 "grep \"pending event get the action\" $WORK_DIR/cdc0-1.log | grep -Eq \"(pendingEventCommitTs[^0-9]*${syncpoint_ts}.*innerAction=0|innerAction=0.*pendingEventCommitTs[^0-9]*${syncpoint_ts})\"" + ensure 60 "grep \"pending event get the action\" $WORK_DIR/cdc0-1.log | grep -Eq \"(pendingEventCommitTs[^0-9]*${syncpoint_ts}.*innerAction=0|innerAction=0.*pendingEventCommitTs[^0-9]*${syncpoint_ts})\"" + + kill $dml_pid || true + wait $dml_pid || true table_id=$(get_table_id "$DB_NAME" "$TABLE_NAME") move_table_with_retry "127.0.0.1:8301" $table_id "$CHANGEFEED_ID" 10 - # The moved dispatcher must start from syncpoint_ts (not syncpoint_ts-1). + # The moved dispatcher must not start from (syncpoint_ts-1), otherwise it may replay events with commitTs <= syncpoint_ts. ensure 30 "grep \"new dispatcher created\" $WORK_DIR/cdc1.log | grep -q \"tableID: ${table_id}\"" dispatcher_start_ts=$(grep "new dispatcher created" $WORK_DIR/cdc1.log | grep "tableID: ${table_id}" | tail -n 1 | grep -oE 'startTs[^0-9]*[0-9]+' | tail -n 1 | grep -oE '[0-9]+' || true) - if [ "$dispatcher_start_ts" != "$syncpoint_ts" ]; then - echo "unexpected dispatcher startTs, got: $dispatcher_start_ts, want: $syncpoint_ts" + if [ "$dispatcher_start_ts" -lt "$syncpoint_ts" ]; then + echo "unexpected dispatcher startTs, got: $dispatcher_start_ts, want >= $syncpoint_ts" exit 1 fi # Wait until the syncpoint is written downstream, then validate snapshot consistency by sync_diff. - ensure 30 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" + ensure 120 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) echo "secondary_ts: $secondary_ts" deployConfig "$syncpoint_ts" "$secondary_ts" - check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 60 + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 120 rm -f $CUR/conf/diff_config.toml export GO_FAILPOINTS='' cleanup_process $CDC_BINARY } -trap 'stop_tidb_cluster; collect_logs $WORK_DIR' EXIT +trap 'if [ "${dml_pid}" -ne 0 ]; then kill "${dml_pid}" >/dev/null 2>&1 || true; fi; stop_tidb_cluster; collect_logs $WORK_DIR' EXIT run $* check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 65ab9c33eb232e278bac436271b5348d038fbb1b Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 19 Dec 2025 10:19:49 +0800 Subject: [PATCH 06/20] update --- tests/integration_tests/ddl_move_table/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/ddl_move_table/run.sh b/tests/integration_tests/ddl_move_table/run.sh index 5308447dbc..5ba4294b3a 100644 --- a/tests/integration_tests/ddl_move_table/run.sh +++ b/tests/integration_tests/ddl_move_table/run.sh @@ -84,7 +84,7 @@ run() { cdc_pid_0=$(get_cdc_pid "$CDC_HOST" "$CDC_PORT") kill_cdc_pid $cdc_pid_0 - export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=sleep(20000)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true);github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=1*sleep(60000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-1" --addr "127.0.0.1:8300" check_coordinator_and_maintainer "127.0.0.1:8300" "$CHANGEFEED_ID" 60 From 3573206d2f3f73a170476ca8166ceee049dd7917 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 19 Dec 2025 10:33:01 +0800 Subject: [PATCH 07/20] update --- .../dispatcher/basic_dispatcher.go | 10 +++++++--- .../dispatcher/redo_dispatcher.go | 2 -- .../dispatcher_manager_redo.go | 9 ++++++++- maintainer/replica/replication_span.go | 18 ++++++++---------- 4 files changed, 23 insertions(+), 16 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 87c5094c8b..5bfca9fa37 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -133,11 +133,15 @@ type BasicDispatcher struct { skipSyncpointAtStartTs bool // skipDMLAsStartTs indicates whether to skip DML events at startTs+1 timestamp. // When true, the dispatcher should filter out DML events with commitTs == startTs+1, but keep DDL events. - // This flag is set to true ONLY when is_syncpoint=false AND finished=0 in ddl-ts table (non-syncpoint DDL not finished). - // In this case, we return startTs = ddlTs-1 to replay the DDL, and skip the already-written DML at ddlTs - // to avoid duplicate writes while ensuring the DDL is replayed. + // This flag is set to true in two secnaios: + // 1. when is_syncpoint=false AND finished=0 in ddl-ts table (non-syncpoint DDL not finished). + // In this case, we return startTs = ddlTs-1 to replay the DDL, and skip the already-written DML at ddlTs + // to avoid duplicate writes while ensuring the DDL is replayed. // Note: When is_syncpoint=true AND finished=0 (DDL finished but syncpoint not finished), // skipDMLAsStartTs is false because the DDL is already completed and DML should be processed normally. + // 2. maintainer ask dispatcher to make a move operator, while the dispatcher just dealing with a ddl event. + // and the block state for ddl event is still waiting. + // In this case, we also return startTs = ddlTs-1 to replay the DDL but skip the dmls. skipDMLAsStartTs bool // The ts from pdClock when the dispatcher is created. // when downstream is mysql-class, for dml event we need to compare the commitTs with this ts diff --git a/downstreamadapter/dispatcher/redo_dispatcher.go b/downstreamadapter/dispatcher/redo_dispatcher.go index f43201ccc0..33db6e3263 100644 --- a/downstreamadapter/dispatcher/redo_dispatcher.go +++ b/downstreamadapter/dispatcher/redo_dispatcher.go @@ -54,8 +54,6 @@ func NewRedoDispatcher( schemaID, schemaIDToDispatchers, skipSyncpointAtStartTs, - // skipDMLAsStartTs is used when a dispatcher is recreated during an in-flight DDL barrier: - // we may start from (blockTs-1) to replay the DDL at blockTs, while skipping DML at blockTs (startTs+1). skipDMLAsStartTs, 0, common.RedoMode, diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go index f3b509a8ac..38c69121a7 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go @@ -128,7 +128,14 @@ func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dis } for idx, id := range dispatcherIds { - skipDMLAsStartTs := skipDMLAsStartTsList[idx] || scheduleSkipDMLAsStartTsList[idx] + // if the newStartTs equals to the original startTs, we need to combine the skipDMLAsStartTs flag + // otherwise, we just use the skipDMLAsStartTs flag from mysql sink + var skipDMLAsStartTs bool + if newStartTsList[idx] == startTsList[idx] { + skipDMLAsStartTs = scheduleSkipDMLAsStartTsList[idx] || skipDMLAsStartTsList[idx] + } else { + skipDMLAsStartTs = skipDMLAsStartTsList[idx] + } rd := dispatcher.NewRedoDispatcher( id, tableSpans[idx], diff --git a/maintainer/replica/replication_span.go b/maintainer/replica/replication_span.go index c03e8ee627..2f24bca1a7 100644 --- a/maintainer/replica/replication_span.go +++ b/maintainer/replica/replication_span.go @@ -229,19 +229,17 @@ func (r *SpanReplication) GetGroupID() replica.GroupID { } func (r *SpanReplication) NewAddDispatcherMessage(server node.ID) *messaging.TargetMessage { - // When a syncpoint is in-flight (WAITING/WRITING), the maintainer will force the span checkpoint to BlockTs-1 - // when handling block status. If we start a moved/recreated dispatcher from that forced checkpoint, it may re-scan - // and re-apply events with commitTs <= BlockTs, which can race with the syncpoint write and corrupt the snapshot - // semantics of that syncpoint. Starting from BlockTs avoids replaying those events, and is safe because the - // dispatcher can only enter the syncpoint barrier after all events with commitTs <= BlockTs have been pushed - // downstream. startTs := r.status.Load().CheckpointTs skipDMLAsStartTs := false - if state := r.blockState.Load(); state != nil && - state.IsBlocked && - (state.Stage == heartbeatpb.BlockStage_WAITING || state.Stage == heartbeatpb.BlockStage_WRITING) && - state.BlockTs > 0 { + if state := r.blockState.Load(); state != nil && state.IsBlocked && + (state.Stage == heartbeatpb.BlockStage_WAITING || state.Stage == heartbeatpb.BlockStage_WRITING) && state.BlockTs > 0 { if state.IsSyncPoint { + // When a syncpoint is in-flight (WAITING/WRITING), the maintainer will force the span checkpoint to BlockTs-1 + // when handling block status. If we start a moved/recreated dispatcher from that forced checkpoint, it may re-scan + // and re-apply events with commitTs <= BlockTs, which can race with the syncpoint write and corrupt the snapshot + // semantics of that syncpoint. Starting from BlockTs avoids replaying those events, and is safe because the + // dispatcher can only enter the syncpoint barrier after all events with commitTs <= BlockTs have been pushed + // downstream. if state.BlockTs > startTs { startTs = state.BlockTs } From 737cc6d37b96a7fccdba2131668ad11af97f650a Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 19 Dec 2025 11:04:04 +0800 Subject: [PATCH 08/20] update --- .../dispatcher/basic_dispatcher.go | 2 +- .../dispatcher_manager_test.go | 51 ------------------- 2 files changed, 1 insertion(+), 52 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 5bfca9fa37..94a8533748 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -133,7 +133,7 @@ type BasicDispatcher struct { skipSyncpointAtStartTs bool // skipDMLAsStartTs indicates whether to skip DML events at startTs+1 timestamp. // When true, the dispatcher should filter out DML events with commitTs == startTs+1, but keep DDL events. - // This flag is set to true in two secnaios: + // This flag is set to true in two scenarios: // 1. when is_syncpoint=false AND finished=0 in ddl-ts table (non-syncpoint DDL not finished). // In this case, we return startTs = ddlTs-1 to replay the DDL, and skip the already-written DML at ddlTs // to avoid duplicate writes while ensuring the DDL is replayed. diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_test.go b/downstreamadapter/dispatchermanager/dispatcher_manager_test.go index 3cdda91e6d..61684288f5 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_test.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_test.go @@ -125,57 +125,6 @@ func createTestManager(t *testing.T) *DispatcherManager { return manager } -func TestNewEventDispatchers_PropagateSkipDMLAsStartTs(t *testing.T) { - manager := createTestManager(t) - manager.metricTableTriggerEventDispatcherCount = metrics.TableTriggerEventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "tableTrigger") - manager.metricCreateDispatcherDuration = metrics.CreateDispatcherDuration.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "eventDispatcher") - - dispatcherID := common.NewDispatcherID() - totalSpan := common.TableIDToComparableSpan(0, 1) - infos := map[common.DispatcherID]dispatcherCreateInfo{ - dispatcherID: { - Id: dispatcherID, - TableSpan: &heartbeatpb.TableSpan{TableID: totalSpan.TableID, StartKey: totalSpan.StartKey, EndKey: totalSpan.EndKey}, - StartTs: 99, - SchemaID: 1, - - SkipDMLAsStartTs: true, - }, - } - require.NoError(t, manager.newEventDispatchers(infos, false)) - - d, ok := manager.dispatcherMap.Get(dispatcherID) - require.True(t, ok) - require.True(t, d.GetSkipDMLAsStartTs()) -} - -func TestNewRedoDispatchers_PropagateSkipDMLAsStartTs(t *testing.T) { - manager := createTestManager(t) - manager.redoDispatcherMap = newDispatcherMap[*dispatcher.RedoDispatcher]() - manager.redoSchemaIDToDispatchers = dispatcher.NewSchemaIDToDispatchers() - manager.redoSink = nil - manager.metricRedoTableTriggerEventDispatcherCount = metrics.TableTriggerEventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") - manager.metricRedoEventDispatcherCount = metrics.EventDispatcherGauge.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") - manager.metricRedoCreateDispatcherDuration = metrics.CreateDispatcherDuration.WithLabelValues(manager.changefeedID.Keyspace(), manager.changefeedID.Name(), "redoDispatcher") - - dispatcherID := common.NewDispatcherID() - infos := map[common.DispatcherID]dispatcherCreateInfo{ - dispatcherID: { - Id: dispatcherID, - TableSpan: common.KeyspaceDDLSpan(0), - StartTs: 99, - SchemaID: 0, - - SkipDMLAsStartTs: true, - }, - } - require.NoError(t, manager.newRedoDispatchers(infos, false)) - - d, ok := manager.redoDispatcherMap.Get(dispatcherID) - require.True(t, ok) - require.True(t, d.GetSkipDMLAsStartTs()) -} - func TestMergeDispatcherNormal(t *testing.T) { manager := createTestManager(t) From 1946dd6b7d84e65bfee190ec534b89a62ccf4ed2 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 23 Dec 2025 15:01:57 +0800 Subject: [PATCH 09/20] update --- .../dispatcher/basic_dispatcher.go | 1 + downstreamadapter/dispatchermanager/task.go | 166 +++++++++++++----- 2 files changed, 126 insertions(+), 41 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 125518bd5d..27cb6c0583 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -72,6 +72,7 @@ type Dispatcher interface { SetTryRemoving() GetHeartBeatInfo(h *HeartBeatInfo) GetComponentStatus() heartbeatpb.ComponentState + GetBlockEventStatus() *heartbeatpb.State GetBlockStatusesChan() chan *heartbeatpb.TableSpanBlockStatus GetEventSizePerSecond() float32 IsTableTriggerEventDispatcher() bool diff --git a/downstreamadapter/dispatchermanager/task.go b/downstreamadapter/dispatchermanager/task.go index a2da8b8fac..e64234a5f7 100644 --- a/downstreamadapter/dispatchermanager/task.go +++ b/downstreamadapter/dispatchermanager/task.go @@ -222,6 +222,7 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch // Step1: close all dispatchers to be merged, calculate the min checkpointTs of the merged dispatcher minCheckpointTs := uint64(math.MaxUint64) closedList := make([]bool, len(t.dispatcherIDs)) // record whether the dispatcher is closed successfully + pendingStates := make([]*heartbeatpb.State, len(t.dispatcherIDs)) closedCount := 0 count := 0 for closedCount < len(t.dispatcherIDs) { @@ -246,6 +247,10 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch if watermark.CheckpointTs < minCheckpointTs { minCheckpointTs = watermark.CheckpointTs } + // Record pending block event status when the source dispatcher is closed successfully. + // This is used to derive a safe startTs for merged dispatcher when all source dispatchers + // are waiting for the same block event (DDL or syncpoint). + pendingStates[idx] = dispatcher.GetBlockEventStatus() closedList[idx] = true closedCount++ } else { @@ -263,59 +268,25 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch ) } - // Step2: set the minCheckpointTs as the startTs of the merged dispatcher, + // Step2: resolve startTs and skip flags for the merged dispatcher, // set the pd clock currentTs as the currentPDTs of the merged dispatcher, // change the component status of the merged dispatcher to Initializing // set dispatcher into dispatcherMap and related field // notify eventCollector to update the merged dispatcher startTs - // - // if the sink is mysql, we need to calculate the real startTs of the merged dispatcher based on minCheckpointTs - // Here is a example to show why we need to calculate the real startTs: - // 1. we have 5 dispatchers of a split-table, and deal with a ts=t1 ddl. - // 2. the ddl is flushed in one dispatcher, but not finish passing in other dispatchers. - // 3. if we don't calculate the real startTs, the final startTs of the merged dispatcher will be t1-x, - // which will lead to the new dispatcher receive the previous dml and ddl, which is not match the new schema, - // leading to writing downstream failed. - // 4. so we need to calculate the real startTs of the merged dispatcher by the tableID based on ddl_ts. - // - // For redo - // We don't need to calculate the true start timestamp (start-ts) because the redo metadata records the minimum checkpoint timestamp and resolved timestamp. - // The merger dispatcher operates by first creating a dispatcher and then removing it. - // Even if the redo dispatcher’s start-ts is less than that of the common dispatcher, we still record the correct redo metadata log. - if common.IsDefaultMode(t.mergedDispatcher.GetMode()) && t.manager.sink.SinkType() == common.MysqlSinkType { - newStartTsList, skipSyncpointAtStartTsList, skipDMLAsStartTsList, err := t.manager.sink.(*mysql.Sink).GetTableRecoveryInfo([]int64{t.mergedDispatcher.GetTableSpan().TableID}, []int64{int64(minCheckpointTs)}, false) - if err != nil { - log.Error("get table recovery info for merge dispatcher failed", - zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), - zap.Stringer("changefeedID", t.manager.changefeedID), - zap.Error(err), - ) - t.mergedDispatcher.HandleError(err) - return - } - log.Info("get table recovery info for Merge Dispatcher", - zap.Stringer("changefeedID", t.manager.changefeedID), - zap.Any("receiveStartTs", minCheckpointTs), - zap.Any("realStartTs", newStartTsList), - zap.Any("skipSyncpointAtStartTsList", skipSyncpointAtStartTsList), - zap.Any("skipDMLAsStartTsList", skipDMLAsStartTsList), - ) - t.mergedDispatcher.SetStartTs(uint64(newStartTsList[0])) - t.mergedDispatcher.SetSkipSyncpointAtStartTs(skipSyncpointAtStartTsList[0]) - t.mergedDispatcher.SetSkipDMLAsStartTs(skipDMLAsStartTsList[0]) - } else { - t.mergedDispatcher.SetStartTs(minCheckpointTs) - } + finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs := resolveMergedDispatcherStartTs(t, minCheckpointTs, pendingStates) + t.mergedDispatcher.SetStartTs(finalStartTs) + t.mergedDispatcher.SetSkipSyncpointAtStartTs(finalSkipSyncpointAtStartTs) + t.mergedDispatcher.SetSkipDMLAsStartTs(finalSkipDMLAsStartTs) t.mergedDispatcher.SetCurrentPDTs(t.manager.pdClock.CurrentTS()) t.mergedDispatcher.SetComponentStatus(heartbeatpb.ComponentState_Initializing) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).CommitAddDispatcher(t.mergedDispatcher, minCheckpointTs) + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).CommitAddDispatcher(t.mergedDispatcher, finalStartTs) log.Info("merge dispatcher commit", zap.Stringer("changefeedID", t.manager.changefeedID), zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), zap.Int64("mode", t.mergedDispatcher.GetMode()), zap.Any("tableSpan", common.FormatTableSpan(t.mergedDispatcher.GetTableSpan())), - zap.Uint64("startTs", minCheckpointTs), + zap.Uint64("startTs", finalStartTs), ) // Step3: cancel the merge task @@ -337,6 +308,119 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch } } +// resolveMergedDispatcherStartTs returns the effective startTs and skip flags for the merged dispatcher. +// +// Inputs: +// - minCheckpointTs: min checkpointTs among all source dispatchers, collected after they are closed. +// - pendingStates: per-source block state from GetBlockEventStatus() captured at close time. +// +// Algorithm: +// 1. Build a merge candidate from minCheckpointTs. +// If all source dispatchers have a non-nil pending block state and they refer to the same (commitTs, isSyncPoint), +// adjust the merge candidate so the merged dispatcher can replay that block event safely: +// - DDL: startTs = commitTs - 1, skipDMLAsStartTs = true. +// - SyncPoint: startTs = commitTs. +// The merge candidate always uses skipSyncpointAtStartTs = false. +// 2. If the sink is MySQL, query downstream ddl_ts recovery info using the merge candidate startTs and merge the results: +// - If recoveryStartTs > mergeStartTsCandidate: use recoveryStartTs and its skip flags. +// - If recoveryStartTs == mergeStartTsCandidate: OR the skip flags. +// - If recoveryStartTs < mergeStartTsCandidate: keep the merge candidate. +// If the query fails, the error is reported via mergedDispatcher.HandleError and the merge candidate is returned. +// +// For non-MySQL and redo, the merge candidate is the final result. +func resolveMergedDispatcherStartTs(t *MergeCheckTask, minCheckpointTs uint64, pendingStates []*heartbeatpb.State) (uint64, bool, bool) { + mergeStartTsCandidate := minCheckpointTs + mergeSkipSyncpointAtStartTsCandidate := false + mergeSkipDMLAsStartTsCandidate := false + + // If all source dispatchers have a pending block event and they are the same one, + // adjust the startTs to ensure the merged dispatcher can replay it safely. + allSamePending := true + var pendingCommitTs uint64 + var pendingIsSyncPoint bool + for idx, state := range pendingStates { + if state == nil { + allSamePending = false + break + } + if idx == 0 { + pendingCommitTs = state.BlockTs + pendingIsSyncPoint = state.IsSyncPoint + continue + } + if state.BlockTs != pendingCommitTs || state.IsSyncPoint != pendingIsSyncPoint { + allSamePending = false + break + } + } + if allSamePending { + if pendingIsSyncPoint { + mergeStartTsCandidate = pendingCommitTs + } else if pendingCommitTs > 0 { + mergeStartTsCandidate = pendingCommitTs - 1 + mergeSkipDMLAsStartTsCandidate = true + } else { + log.Warn("pending ddl has zero commit ts, fallback to min checkpoint ts", + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Uint64("minCheckpointTs", minCheckpointTs), + zap.Any("mergedDispatcher", t.mergedDispatcher.GetId())) + } + log.Info("merge dispatcher uses pending block event to calculate start ts", + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Any("mergedDispatcher", t.mergedDispatcher.GetId()), + zap.Uint64("pendingCommitTs", pendingCommitTs), + zap.Bool("pendingIsSyncPoint", pendingIsSyncPoint), + zap.Uint64("startTs", mergeStartTsCandidate), + zap.Bool("skipSyncpointAtStartTs", mergeSkipSyncpointAtStartTsCandidate), + zap.Bool("skipDMLAsStartTs", mergeSkipDMLAsStartTsCandidate)) + } + + finalStartTs := mergeStartTsCandidate + finalSkipSyncpointAtStartTs := mergeSkipSyncpointAtStartTsCandidate + finalSkipDMLAsStartTs := mergeSkipDMLAsStartTsCandidate + + if common.IsDefaultMode(t.mergedDispatcher.GetMode()) && t.manager.sink.SinkType() == common.MysqlSinkType { + newStartTsList, skipSyncpointAtStartTsList, skipDMLAsStartTsList, err := t.manager.sink.(*mysql.Sink).GetTableRecoveryInfo( + []int64{t.mergedDispatcher.GetTableSpan().TableID}, + []int64{int64(mergeStartTsCandidate)}, + false, + ) + if err != nil { + log.Error("get table recovery info for merge dispatcher failed", + zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Error(err), + ) + t.mergedDispatcher.HandleError(err) + return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs + } + recoveryStartTs := uint64(newStartTsList[0]) + recoverySkipSyncpointAtStartTs := skipSyncpointAtStartTsList[0] + recoverySkipDMLAsStartTs := skipDMLAsStartTsList[0] + if recoveryStartTs > mergeStartTsCandidate { + finalStartTs = recoveryStartTs + finalSkipSyncpointAtStartTs = recoverySkipSyncpointAtStartTs + finalSkipDMLAsStartTs = recoverySkipDMLAsStartTs + } else if recoveryStartTs == mergeStartTsCandidate { + finalSkipSyncpointAtStartTs = mergeSkipSyncpointAtStartTsCandidate || recoverySkipSyncpointAtStartTs + finalSkipDMLAsStartTs = mergeSkipDMLAsStartTsCandidate || recoverySkipDMLAsStartTs + } + + log.Info("get table recovery info for merge dispatcher", + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Uint64("mergeStartTsCandidate", mergeStartTsCandidate), + zap.Any("recoveryStartTs", newStartTsList), + zap.Any("recoverySkipSyncpointAtStartTsList", skipSyncpointAtStartTsList), + zap.Any("recoverySkipDMLAsStartTsList", skipDMLAsStartTsList), + zap.Uint64("finalStartTs", finalStartTs), + zap.Bool("finalSkipSyncpointAtStartTs", finalSkipSyncpointAtStartTs), + zap.Bool("finalSkipDMLAsStartTs", finalSkipDMLAsStartTs), + ) + } + + return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs +} + var ( removeDispatcherTaskSchedulerOnce sync.Once removeDispatcherTaskScheduler threadpool.ThreadPool From 38dd720e9e1972ebec7227e1b100c2ac6dd0d381 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 23 Dec 2025 19:13:26 +0800 Subject: [PATCH 10/20] update --- tests/integration_tests/ddl_move_table/run.sh | 68 ++++++++++++++++++- .../syncpoint_move_table/conf/changefeed.toml | 4 ++ .../syncpoint_move_table/run.sh | 47 ++++++++++++- 3 files changed, 113 insertions(+), 6 deletions(-) diff --git a/tests/integration_tests/ddl_move_table/run.sh b/tests/integration_tests/ddl_move_table/run.sh index 5ba4294b3a..563c11e11a 100644 --- a/tests/integration_tests/ddl_move_table/run.sh +++ b/tests/integration_tests/ddl_move_table/run.sh @@ -62,7 +62,13 @@ run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - do_retry 5 3 cdc_cli_changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" -c "$CHANGEFEED_ID" + cat >$WORK_DIR/changefeed.toml <= 2' >/dev/null" + + export GO_FAILPOINTS='github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=pause;github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforePass=pause' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" + move_split_table_with_retry "127.0.0.1:8302" $table_1_id "$CHANGEFEED_ID" 10 + + merge_log=$WORK_DIR/cdc2.log + touch $merge_log + merge_log_offset=$(wc -l <"$merge_log") + run_sql "ALTER TABLE ${DB_NAME}.${TABLE_1_NEW} ADD COLUMN c_merge INT DEFAULT 0;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + ensure 60 "tail -n +$((merge_log_offset + 1)) \"$merge_log\" | grep -q \"pending event get the action\"" + ddl_merge_ts=$(tail -n +$((merge_log_offset + 1)) "$merge_log" | grep "pending event get the action" | head -n 1 | grep -oE 'pendingEventCommitTs[^0-9]*[0-9]+' | head -n 1 | grep -oE '[0-9]+' || true) + if [ -z "$ddl_merge_ts" ]; then + echo "failed to extract DDL commitTs for merge from logs" + exit 1 + fi + expected_merge_start_ts=$((ddl_merge_ts - 1)) + + merge_table_with_retry $table_1_id "$CHANGEFEED_ID" 10 + ensure 60 "grep -q \"merge dispatcher uses pending block event to calculate start ts\" \"$merge_log\"" + merge_line=$(grep "merge dispatcher uses pending block event to calculate start ts" "$merge_log" | grep -E "pendingCommitTs[^0-9]*${ddl_merge_ts}" | tail -n 1 || true) + if [ -z "$merge_line" ]; then + echo "failed to find merge startTs decision log line" + exit 1 + fi + merge_start_ts=$(echo "$merge_line" | grep -oE 'startTs[^0-9]*[0-9]+' | tail -n 1 | grep -oE '[0-9]+' || true) + merge_pending_is_syncpoint=$(echo "$merge_line" | grep -oE 'pendingIsSyncPoint[^a-zA-Z]*(true|false)' | tail -n 1 | grep -oE '(true|false)' | tail -n 1 || true) + merge_skip_dml=$(echo "$merge_line" | grep -oE 'skipDMLAsStartTs[^a-zA-Z]*(true|false)' | tail -n 1 | grep -oE '(true|false)' | tail -n 1 || true) + if [ "$merge_start_ts" != "$expected_merge_start_ts" ]; then + echo "unexpected merged dispatcher startTs, got: $merge_start_ts, want: $expected_merge_start_ts" + exit 1 + fi + if [ "$merge_pending_is_syncpoint" != "false" ]; then + echo "unexpected pendingIsSyncPoint, got: $merge_pending_is_syncpoint, want: false" + exit 1 + fi + if [ "$merge_skip_dml" != "true" ]; then + echo "unexpected merged dispatcher skipDMLAsStartTs, got: $merge_skip_dml, want: true" + exit 1 + fi + + cdc_pid_2=$(get_cdc_pid "127.0.0.1" "8302") + kill_cdc_pid $cdc_pid_2 + export GO_FAILPOINTS='' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2-1" --addr "127.0.0.1:8302" + ensure 120 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT COUNT(*) FROM information_schema.columns WHERE table_schema='${DB_NAME}' AND table_name='${TABLE_1_NEW}' AND column_name='c_merge';\" | grep -q '^1$'" + deployDiffConfig check_sync_diff $WORK_DIR $WORK_DIR/diff_config.toml 60 rm -f $WORK_DIR/diff_config.toml @@ -143,7 +205,7 @@ run() { cleanup_process $CDC_BINARY } -trap 'stop_tidb_cluster; collect_logs $WORK_DIR' EXIT +trap 'stop_test $WORK_DIR' EXIT run $* check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml b/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml index c21ae7ca60..e83b0917c7 100644 --- a/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml +++ b/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml @@ -1,3 +1,7 @@ enable-sync-point = true sync-point-interval = "30s" +[scheduler] +enable-table-across-nodes = true +region-threshold=1 +region-count-per-span=10 diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/syncpoint_move_table/run.sh index f5650ace42..081602a66a 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/syncpoint_move_table/run.sh @@ -16,6 +16,7 @@ SINK_TYPE=$1 DB_NAME="sp_move" TABLE_NAME="t" +MERGE_TABLE_NAME="t_merge" CHANGEFEED_ID="test" dml_pid=0 @@ -41,7 +42,13 @@ run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - do_retry 5 3 cdc_cli_changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" -c "$CHANGEFEED_ID" + cat >$WORK_DIR/changefeed_scheduler.toml </dev/null 2>&1 || true; fi; stop_tidb_cluster; collect_logs $WORK_DIR' EXIT +trap 'stop_test $WORK_DIR' EXIT run $* check_logs $WORK_DIR echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From ac507c093bce393035bc297d4c59e7711ebce7ec Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 23 Dec 2025 23:42:40 +0800 Subject: [PATCH 11/20] update --- .../dispatcher/basic_dispatcher.go | 42 ++++++++++--------- 1 file changed, 23 insertions(+), 19 deletions(-) diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index 27cb6c0583..afdf302ac5 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -614,25 +614,7 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D actionCommitTs := action.CommitTs actionIsSyncPoint := action.IsSyncPoint d.sharedInfo.GetBlockEventExecutor().Submit(d, func() { - failpoint.Inject("BlockOrWaitBeforeWrite", nil) - err := d.AddBlockEventToSink(pendingEvent) - if err != nil { - d.HandleError(err) - return - } - failpoint.Inject("BlockOrWaitReportAfterWrite", nil) - - d.sharedInfo.blockStatusesChan <- &heartbeatpb.TableSpanBlockStatus{ - ID: d.id.ToPB(), - State: &heartbeatpb.State{ - IsBlocked: true, - BlockTs: actionCommitTs, - IsSyncPoint: actionIsSyncPoint, - Stage: heartbeatpb.BlockStage_DONE, - }, - Mode: d.GetMode(), - } - GetDispatcherStatusDynamicStream().Wake(d.id) + d.ExecutePendingDDL(pendingEvent, actionCommitTs, actionIsSyncPoint) }) return true } else { @@ -666,6 +648,28 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D return false } +func (d *BasicDispatcher) ExecutePendingDDL(pendingEvent commonEvent.BlockEvent, actionCommitTs uint64, actionIsSyncPoint bool) { + failpoint.Inject("BlockOrWaitBeforeWrite", nil) + err := d.AddBlockEventToSink(pendingEvent) + if err != nil { + d.HandleError(err) + return + } + failpoint.Inject("BlockOrWaitReportAfterWrite", nil) + + d.sharedInfo.blockStatusesChan <- &heartbeatpb.TableSpanBlockStatus{ + ID: d.id.ToPB(), + State: &heartbeatpb.State{ + IsBlocked: true, + BlockTs: actionCommitTs, + IsSyncPoint: actionIsSyncPoint, + Stage: heartbeatpb.BlockStage_DONE, + }, + Mode: d.GetMode(), + } + GetDispatcherStatusDynamicStream().Wake(d.id) +} + // shouldBlock check whether the event should be blocked(to wait maintainer response) // For the ddl event with more than one blockedTable, it should block. // For the ddl event with only one blockedTable, it should block only if the table is not complete span. From dd08c0ba067f17aa9a97bac97c8c1d1fb5c0640e Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 24 Dec 2025 17:40:07 +0800 Subject: [PATCH 12/20] update --- maintainer/barrier_event.go | 27 ++++++++++++++++--- maintainer/replica/replication_span.go | 4 +++ .../_utils/query_dispatcher_count | 6 ++++- .../run.sh | 4 +-- .../conf/changefeed.toml | 0 .../conf/diff_config_part1.toml | 2 +- .../conf/diff_config_part2.toml | 0 .../run.sh | 24 +++++++++-------- tests/integration_tests/run_heavy_it_in_ci.sh | 4 +-- 9 files changed, 51 insertions(+), 20 deletions(-) rename tests/integration_tests/{ddl_move_table => in_flight_ddl_during_scheduling}/run.sh (97%) rename tests/integration_tests/{syncpoint_move_table => in_flight_syncpoint_during_sheduling}/conf/changefeed.toml (100%) rename tests/integration_tests/{syncpoint_move_table => in_flight_syncpoint_during_sheduling}/conf/diff_config_part1.toml (77%) rename tests/integration_tests/{syncpoint_move_table => in_flight_syncpoint_during_sheduling}/conf/diff_config_part2.toml (100%) rename tests/integration_tests/{syncpoint_move_table => in_flight_syncpoint_during_sheduling}/run.sh (93%) diff --git a/maintainer/barrier_event.go b/maintainer/barrier_event.go index d9be1e07b3..7437d0995b 100644 --- a/maintainer/barrier_event.go +++ b/maintainer/barrier_event.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/ticdc/heartbeatpb" "github.com/pingcap/ticdc/maintainer/operator" "github.com/pingcap/ticdc/maintainer/range_checker" + "github.com/pingcap/ticdc/maintainer/replica" "github.com/pingcap/ticdc/maintainer/span" "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" @@ -474,7 +475,7 @@ func (be *BarrierEvent) checkBlockedDispatchers() { for _, tableId := range be.blockedDispatchers.TableIDs { replications := be.spanController.GetTasksByTableID(tableId) for _, replication := range replications { - if replication.GetStatus().CheckpointTs >= be.commitTs { + if forwardBarrierEvent(replication, be) { // one related table has forward checkpointTs, means the block event can be advanced be.selected.Store(true) be.writerDispatcherAdvanced = true @@ -493,7 +494,7 @@ func (be *BarrierEvent) checkBlockedDispatchers() { schemaID := be.blockedDispatchers.SchemaID replications := be.spanController.GetTasksBySchemaID(schemaID) for _, replication := range replications { - if replication.GetStatus().CheckpointTs >= be.commitTs { + if forwardBarrierEvent(replication, be) { // one related table has forward checkpointTs, means the block event can be advanced be.selected.Store(true) be.writerDispatcherAdvanced = true @@ -510,7 +511,7 @@ func (be *BarrierEvent) checkBlockedDispatchers() { case heartbeatpb.InfluenceType_All: replications := be.spanController.GetAllTasks() for _, replication := range replications { - if replication.GetStatus().CheckpointTs >= be.commitTs { + if forwardBarrierEvent(replication, be) { // one related table has forward checkpointTs, means the block event can be advanced be.selected.Store(true) be.writerDispatcherAdvanced = true @@ -526,6 +527,26 @@ func (be *BarrierEvent) checkBlockedDispatchers() { } } +// forwardBarrierEvent checks whether the barrier event can be forwarded for the given replication +func forwardBarrierEvent(replication *replica.SpanReplication, event *BarrierEvent) bool { + if replication.GetStatus().CheckpointTs > event.commitTs { + return true + } + + blockState := replication.GetBlockState() + if blockState != nil { + if blockState.BlockTs > event.commitTs { + return true + } else if blockState.BlockTs == event.commitTs { + // if replication is syncpoint, but event is not syncpoint, we can forward the event + if blockState.IsSyncPoint && !event.isSyncPoint { + return true + } + } + } + return false +} + func (be *BarrierEvent) resend(mode int64) []*messaging.TargetMessage { if time.Since(be.lastResendTime) < time.Second { return nil diff --git a/maintainer/replica/replication_span.go b/maintainer/replica/replication_span.go index 2f24bca1a7..b7e353f56e 100644 --- a/maintainer/replica/replication_span.go +++ b/maintainer/replica/replication_span.go @@ -193,6 +193,10 @@ func (r *SpanReplication) UpdateBlockState(newState heartbeatpb.State) { r.blockState.Store(&newState) } +func (r *SpanReplication) GetBlockState() *heartbeatpb.State { + return r.blockState.Load() +} + func (r *SpanReplication) GetSchemaID() int64 { return r.schemaID } diff --git a/tests/integration_tests/_utils/query_dispatcher_count b/tests/integration_tests/_utils/query_dispatcher_count index c96a40802b..a7eb414b7b 100755 --- a/tests/integration_tests/_utils/query_dispatcher_count +++ b/tests/integration_tests/_utils/query_dispatcher_count @@ -3,7 +3,7 @@ # parameter 2: changefeed id # parameter 3: target count, if target count is -1, means the target count is not important, just not to be null # parameter 4: retry count -# parameter 5: comparison mode, if set to "le" (less than or equal), compare value <= target; otherwise compare value == target +# parameter 5: comparison mode, support le, eq, ge, default is eq set -ex @@ -31,6 +31,10 @@ while [[ $count -lt $retryCount ]]; do if [ "$value" -le "$target" ]; then exit 0 fi + elif [ "$comparisonMode" == "ge" ]; then + if [ "$value" -ge "$target" ]; then + exit 0 + fi else if [ "$value" == "$target" ]; then exit 0 diff --git a/tests/integration_tests/ddl_move_table/run.sh b/tests/integration_tests/in_flight_ddl_during_scheduling/run.sh similarity index 97% rename from tests/integration_tests/ddl_move_table/run.sh rename to tests/integration_tests/in_flight_ddl_during_scheduling/run.sh index 563c11e11a..f4a528ec16 100644 --- a/tests/integration_tests/ddl_move_table/run.sh +++ b/tests/integration_tests/in_flight_ddl_during_scheduling/run.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# This test verifies that moving a table dispatcher during an in-flight multi-table DDL barrier +# This test verifies that moving a dispatcher/merge a dispatcher during an in-flight multi-table DDL barrier # does not cause the moved dispatcher to miss the DDL, and that the recreated dispatcher starts # from (blockTs-1) with skipDMLAsStartTs enabled. @@ -151,7 +151,7 @@ EOF # The merged dispatcher must start from (ddl_ts - 1) and enable skipDMLAsStartTs to safely replay the DDL at ddl_ts. run_sql "SPLIT TABLE ${DB_NAME}.${TABLE_1_NEW} BETWEEN (1) AND (100000) REGIONS 20;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} split_table_with_retry $table_1_id "$CHANGEFEED_ID" 10 - ensure 60 "curl -s \"http://127.0.0.1:8300/api/v2/changefeeds/${CHANGEFEED_ID}/tables?mode=0&keyspace=${KEYSPACE_NAME}\" | jq -e '[.[].table_ids[] | select(. == ${table_1_id})] | length >= 2' >/dev/null" + query_dispatcher_count "127.0.0.1:8300" "$CHANGEFEED_ID" 3 100 ge export GO_FAILPOINTS='github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforeWrite=pause;github.com/pingcap/ticdc/downstreamadapter/dispatcher/BlockOrWaitBeforePass=pause' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" diff --git a/tests/integration_tests/syncpoint_move_table/conf/changefeed.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml similarity index 100% rename from tests/integration_tests/syncpoint_move_table/conf/changefeed.toml rename to tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml diff --git a/tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml similarity index 77% rename from tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml rename to tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml index c1967b6bde..a464454bb0 100644 --- a/tests/integration_tests/syncpoint_move_table/conf/diff_config_part1.toml +++ b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/tidb_cdc_test/syncpoint_move_table/sync_diff/output" + output-dir = "/tmp/tidb_cdc_test/in_flight_syncpoint_during_sheduling/sync_diff/output" source-instances = ["tidb0"] diff --git a/tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml similarity index 100% rename from tests/integration_tests/syncpoint_move_table/conf/diff_config_part2.toml rename to tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml diff --git a/tests/integration_tests/syncpoint_move_table/run.sh b/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh similarity index 93% rename from tests/integration_tests/syncpoint_move_table/run.sh rename to tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh index 081602a66a..22833362ce 100644 --- a/tests/integration_tests/syncpoint_move_table/run.sh +++ b/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh @@ -1,6 +1,6 @@ #!/bin/bash # -# This test verifies that moving a table dispatcher during an in-flight syncpoint does not +# This test verifies that moving a dispatcher/merging a dispatcher during an in-flight syncpoint does not # restart the dispatcher from (syncpoint_ts - 1). Otherwise, the dispatcher may re-scan and # re-apply events with commitTs <= syncpoint_ts while the table-trigger dispatcher is writing # the syncpoint, which can break the snapshot consistency semantics. @@ -22,7 +22,7 @@ dml_pid=0 deployConfig() { cat $CUR/conf/diff_config_part1.toml >$CUR/conf/diff_config.toml - sed -i "s#output-dir = \"/tmp/tidb_cdc_test/syncpoint_move_table/sync_diff/output\"#output-dir = \"$WORK_DIR/sync_diff/output\"#g" $CUR/conf/diff_config.toml + sed -i "s#output-dir = \"/tmp/tidb_cdc_test/in_flight_syncpoint_during_sheduling/sync_diff/output\"#output-dir = \"$WORK_DIR/sync_diff/output\"#g" $CUR/conf/diff_config.toml echo "snapshot = \"$1\"" >>$CUR/conf/diff_config.toml cat $CUR/conf/diff_config_part2.toml >>$CUR/conf/diff_config.toml echo "snapshot = \"$2\"" >>$CUR/conf/diff_config.toml @@ -58,6 +58,16 @@ EOF run_sql "INSERT INTO ${DB_NAME}.${TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "${DB_NAME}.${TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + # Prepare a split table for merge test. It should be blocked by syncpoint together with other tables. + run_sql "CREATE TABLE ${DB_NAME}.${MERGE_TABLE_NAME} (id INT PRIMARY KEY, v INT);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO ${DB_NAME}.${MERGE_TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + check_table_exists "${DB_NAME}.${MERGE_TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 + + merge_table_id=$(get_table_id "$DB_NAME" "$MERGE_TABLE_NAME") + run_sql "SPLIT TABLE ${DB_NAME}.${MERGE_TABLE_NAME} BETWEEN (1) AND (100000) REGIONS 20;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + split_table_with_retry $merge_table_id "$CHANGEFEED_ID" 10 + query_dispatcher_count "127.0.0.1:8300" "$CHANGEFEED_ID" 3 100 ge + # Restart node1 to enable failpoints: # - StopBalanceScheduler: keep the table on node1 until we explicitly move it. # - BlockOrWaitBeforeWrite: block syncpoint writing on the table-trigger dispatcher. @@ -75,16 +85,8 @@ EOF cdc_cli_changefeed update --config="$CUR/conf/changefeed.toml" --changefeed-id="$CHANGEFEED_ID" --no-confirm cdc_cli_changefeed resume --changefeed-id="$CHANGEFEED_ID" - # Prepare a split table for merge test. It should be blocked by syncpoint together with other tables. - run_sql "CREATE TABLE ${DB_NAME}.${MERGE_TABLE_NAME} (id INT PRIMARY KEY, v INT);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - run_sql "INSERT INTO ${DB_NAME}.${MERGE_TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - check_table_exists "${DB_NAME}.${MERGE_TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 - merge_table_id=$(get_table_id "$DB_NAME" "$MERGE_TABLE_NAME") - run_sql "SPLIT TABLE ${DB_NAME}.${MERGE_TABLE_NAME} BETWEEN (1) AND (100000) REGIONS 20;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - split_table_with_retry $merge_table_id "$CHANGEFEED_ID" 10 - query_dispatcher_count "127.0.0.1:8300" "$CHANGEFEED_ID" 22 100 - # Start node2 for moving the table. + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" # Keep generating DML until a syncpoint is triggered. diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index 098f4f5204..d4bce15cfe 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -36,9 +36,9 @@ mysql_groups=( # G02 'ddl_for_split_tables_with_failover' # G03 - 'cdc move_table ddl_move_table checkpoint_race_ddl_crash' + 'cdc move_table in_flight_ddl_during_scheduling checkpoint_race_ddl_crash' # G04 - 'complex_transaction syncpoint syncpoint_move_table syncpoint_check_ts random_drop_message' + 'complex_transaction syncpoint in_flight_syncpoint_during_sheduling syncpoint_check_ts random_drop_message' # G05 'ddl_for_split_tables_with_merge_and_split' # G06 From 0a8242257dbe1c94b5c41a8cac73730e034f2d51 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 24 Dec 2025 17:41:10 +0800 Subject: [PATCH 13/20] update --- .../in_flight_syncpoint_during_sheduling/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh b/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh index 22833362ce..5aff9d21b8 100644 --- a/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh +++ b/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh @@ -58,7 +58,7 @@ EOF run_sql "INSERT INTO ${DB_NAME}.${TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "${DB_NAME}.${TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 - # Prepare a split table for merge test. It should be blocked by syncpoint together with other tables. + # Prepare a split table for merge test. It should be blocked by syncpoint together with other tables. run_sql "CREATE TABLE ${DB_NAME}.${MERGE_TABLE_NAME} (id INT PRIMARY KEY, v INT);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO ${DB_NAME}.${MERGE_TABLE_NAME} VALUES (1, 1), (2, 2), (3, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "${DB_NAME}.${MERGE_TABLE_NAME}" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 60 From fd11dee31ebec78553e7f10cf07db659b2c30427 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 11:54:46 +0800 Subject: [PATCH 14/20] update --- .../dispatchermanager/dispatcher_manager.go | 14 +- .../dispatcher_manager_helper.go | 10 + .../dispatcher_manager_redo.go | 14 +- downstreamadapter/dispatchermanager/task.go | 192 +++++++++++------- maintainer/barrier_event.go | 10 +- 5 files changed, 143 insertions(+), 97 deletions(-) diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager.go b/downstreamadapter/dispatchermanager/dispatcher_manager.go index 1b4f56615e..cd3efd6a71 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager.go @@ -425,14 +425,12 @@ func (e *DispatcherManager) newEventDispatchers(infos map[common.DispatcherID]di } for idx, id := range dispatcherIds { - // if the newStartTs equals to the original startTs, we need to combine the skipDMLAsStartTs flag - // otherwise, we just use the skipDMLAsStartTs flag from mysql sink - var skipDMLAsStartTs bool - if newStartTsList[idx] == startTsList[idx] { - skipDMLAsStartTs = scheduleSkipDMLAsStartTsList[idx] || skipDMLAsStartTsList[idx] - } else { - skipDMLAsStartTs = skipDMLAsStartTsList[idx] - } + skipDMLAsStartTs := resolveSkipDMLAsStartTs( + newStartTsList[idx], + startTsList[idx], + scheduleSkipDMLAsStartTsList[idx], + skipDMLAsStartTsList[idx], + ) d := dispatcher.NewEventDispatcher( id, tableSpans[idx], diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go b/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go index 4046f29959..231e45fba8 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_helper.go @@ -95,6 +95,16 @@ func prepareCreateDispatcher[T dispatcher.Dispatcher](infos map[common.Dispatche return dispatcherIds, tableIds, startTsList, tableSpans, schemaIds, skipDMLAsStartTsList } +func resolveSkipDMLAsStartTs(newStartTs, originalStartTs int64, scheduleSkipDMLAsStartTs, sinkSkipDMLAsStartTs bool) bool { + // When the sink keeps startTs unchanged, preserve the skipDMLAsStartTs decision carried by the + // scheduling request (e.g. recreating a dispatcher during an in-flight DDL barrier). + // If the sink adjusts startTs (e.g. based on ddl_ts recovery), the sink decision dominates. + if newStartTs == originalStartTs { + return scheduleSkipDMLAsStartTs || sinkSkipDMLAsStartTs + } + return sinkSkipDMLAsStartTs +} + func prepareMergeDispatcher[T dispatcher.Dispatcher](changefeedID common.ChangeFeedID, dispatcherIDs []common.DispatcherID, dispatcherMap *DispatcherMap[T], diff --git a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go index 9be8c6f1f6..0be156b3a0 100644 --- a/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go +++ b/downstreamadapter/dispatchermanager/dispatcher_manager_redo.go @@ -150,14 +150,12 @@ func (e *DispatcherManager) newRedoDispatchers(infos map[common.DispatcherID]dis } for idx, id := range dispatcherIds { - // if the newStartTs equals to the original startTs, we need to combine the skipDMLAsStartTs flag - // otherwise, we just use the skipDMLAsStartTs flag from mysql sink - var skipDMLAsStartTs bool - if newStartTsList[idx] == startTsList[idx] { - skipDMLAsStartTs = scheduleSkipDMLAsStartTsList[idx] || skipDMLAsStartTsList[idx] - } else { - skipDMLAsStartTs = skipDMLAsStartTsList[idx] - } + skipDMLAsStartTs := resolveSkipDMLAsStartTs( + newStartTsList[idx], + startTsList[idx], + scheduleSkipDMLAsStartTsList[idx], + skipDMLAsStartTsList[idx], + ) rd := dispatcher.NewRedoDispatcher( id, tableSpans[idx], diff --git a/downstreamadapter/dispatchermanager/task.go b/downstreamadapter/dispatchermanager/task.go index e64234a5f7..b5a8cb47e1 100644 --- a/downstreamadapter/dispatchermanager/task.go +++ b/downstreamadapter/dispatchermanager/task.go @@ -308,6 +308,110 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch } } +type mergedStartTsCandidate struct { + startTs uint64 + skipSyncpointAtStartTs bool + skipDMLAsStartTs bool + + allSamePending bool + pendingCommitTs uint64 + pendingIsSyncPoint bool +} + +func buildMergedStartTsCandidate(minCheckpointTs uint64, pendingStates []*heartbeatpb.State) mergedStartTsCandidate { + candidate := mergedStartTsCandidate{ + startTs: minCheckpointTs, + skipSyncpointAtStartTs: false, + skipDMLAsStartTs: false, + allSamePending: true, + } + + if len(pendingStates) == 0 { + candidate.allSamePending = false + return candidate + } + + var pendingCommitTs uint64 + var pendingIsSyncPoint bool + for idx, state := range pendingStates { + if state == nil { + candidate.allSamePending = false + break + } + if idx == 0 { + pendingCommitTs = state.BlockTs + pendingIsSyncPoint = state.IsSyncPoint + continue + } + if state.BlockTs != pendingCommitTs || state.IsSyncPoint != pendingIsSyncPoint { + candidate.allSamePending = false + break + } + } + candidate.pendingCommitTs = pendingCommitTs + candidate.pendingIsSyncPoint = pendingIsSyncPoint + + if candidate.allSamePending { + if pendingIsSyncPoint { + candidate.startTs = pendingCommitTs + } else if pendingCommitTs > 0 { + candidate.startTs = pendingCommitTs - 1 + candidate.skipDMLAsStartTs = true + } + } + return candidate +} + +func mergeMergedStartTsCandidateWithMySQLRecovery(t *MergeCheckTask, candidate mergedStartTsCandidate) (uint64, bool, bool) { + finalStartTs := candidate.startTs + finalSkipSyncpointAtStartTs := candidate.skipSyncpointAtStartTs + finalSkipDMLAsStartTs := candidate.skipDMLAsStartTs + + if !common.IsDefaultMode(t.mergedDispatcher.GetMode()) || t.manager.sink.SinkType() != common.MysqlSinkType { + return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs + } + + newStartTsList, skipSyncpointAtStartTsList, skipDMLAsStartTsList, err := t.manager.sink.(*mysql.Sink).GetTableRecoveryInfo( + []int64{t.mergedDispatcher.GetTableSpan().TableID}, + []int64{int64(candidate.startTs)}, + false, + ) + if err != nil { + log.Error("get table recovery info for merge dispatcher failed", + zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Error(err), + ) + t.mergedDispatcher.HandleError(err) + return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs + } + + recoveryStartTs := uint64(newStartTsList[0]) + recoverySkipSyncpointAtStartTs := skipSyncpointAtStartTsList[0] + recoverySkipDMLAsStartTs := skipDMLAsStartTsList[0] + if recoveryStartTs > candidate.startTs { + finalStartTs = recoveryStartTs + finalSkipSyncpointAtStartTs = recoverySkipSyncpointAtStartTs + finalSkipDMLAsStartTs = recoverySkipDMLAsStartTs + } else if recoveryStartTs == candidate.startTs { + finalSkipSyncpointAtStartTs = candidate.skipSyncpointAtStartTs || recoverySkipSyncpointAtStartTs + finalSkipDMLAsStartTs = candidate.skipDMLAsStartTs || recoverySkipDMLAsStartTs + } + + log.Info("get table recovery info for merge dispatcher", + zap.Stringer("changefeedID", t.manager.changefeedID), + zap.Uint64("mergeStartTsCandidate", candidate.startTs), + zap.Any("recoveryStartTs", newStartTsList), + zap.Any("recoverySkipSyncpointAtStartTsList", skipSyncpointAtStartTsList), + zap.Any("recoverySkipDMLAsStartTsList", skipDMLAsStartTsList), + zap.Uint64("finalStartTs", finalStartTs), + zap.Bool("finalSkipSyncpointAtStartTs", finalSkipSyncpointAtStartTs), + zap.Bool("finalSkipDMLAsStartTs", finalSkipDMLAsStartTs), + ) + + return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs +} + // resolveMergedDispatcherStartTs returns the effective startTs and skip flags for the merged dispatcher. // // Inputs: @@ -329,37 +433,9 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch // // For non-MySQL and redo, the merge candidate is the final result. func resolveMergedDispatcherStartTs(t *MergeCheckTask, minCheckpointTs uint64, pendingStates []*heartbeatpb.State) (uint64, bool, bool) { - mergeStartTsCandidate := minCheckpointTs - mergeSkipSyncpointAtStartTsCandidate := false - mergeSkipDMLAsStartTsCandidate := false - - // If all source dispatchers have a pending block event and they are the same one, - // adjust the startTs to ensure the merged dispatcher can replay it safely. - allSamePending := true - var pendingCommitTs uint64 - var pendingIsSyncPoint bool - for idx, state := range pendingStates { - if state == nil { - allSamePending = false - break - } - if idx == 0 { - pendingCommitTs = state.BlockTs - pendingIsSyncPoint = state.IsSyncPoint - continue - } - if state.BlockTs != pendingCommitTs || state.IsSyncPoint != pendingIsSyncPoint { - allSamePending = false - break - } - } - if allSamePending { - if pendingIsSyncPoint { - mergeStartTsCandidate = pendingCommitTs - } else if pendingCommitTs > 0 { - mergeStartTsCandidate = pendingCommitTs - 1 - mergeSkipDMLAsStartTsCandidate = true - } else { + candidate := buildMergedStartTsCandidate(minCheckpointTs, pendingStates) + if candidate.allSamePending { + if !candidate.pendingIsSyncPoint && candidate.pendingCommitTs == 0 { log.Warn("pending ddl has zero commit ts, fallback to min checkpoint ts", zap.Stringer("changefeedID", t.manager.changefeedID), zap.Uint64("minCheckpointTs", minCheckpointTs), @@ -368,57 +444,15 @@ func resolveMergedDispatcherStartTs(t *MergeCheckTask, minCheckpointTs uint64, p log.Info("merge dispatcher uses pending block event to calculate start ts", zap.Stringer("changefeedID", t.manager.changefeedID), zap.Any("mergedDispatcher", t.mergedDispatcher.GetId()), - zap.Uint64("pendingCommitTs", pendingCommitTs), - zap.Bool("pendingIsSyncPoint", pendingIsSyncPoint), - zap.Uint64("startTs", mergeStartTsCandidate), - zap.Bool("skipSyncpointAtStartTs", mergeSkipSyncpointAtStartTsCandidate), - zap.Bool("skipDMLAsStartTs", mergeSkipDMLAsStartTsCandidate)) - } - - finalStartTs := mergeStartTsCandidate - finalSkipSyncpointAtStartTs := mergeSkipSyncpointAtStartTsCandidate - finalSkipDMLAsStartTs := mergeSkipDMLAsStartTsCandidate - - if common.IsDefaultMode(t.mergedDispatcher.GetMode()) && t.manager.sink.SinkType() == common.MysqlSinkType { - newStartTsList, skipSyncpointAtStartTsList, skipDMLAsStartTsList, err := t.manager.sink.(*mysql.Sink).GetTableRecoveryInfo( - []int64{t.mergedDispatcher.GetTableSpan().TableID}, - []int64{int64(mergeStartTsCandidate)}, - false, - ) - if err != nil { - log.Error("get table recovery info for merge dispatcher failed", - zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), - zap.Stringer("changefeedID", t.manager.changefeedID), - zap.Error(err), - ) - t.mergedDispatcher.HandleError(err) - return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs - } - recoveryStartTs := uint64(newStartTsList[0]) - recoverySkipSyncpointAtStartTs := skipSyncpointAtStartTsList[0] - recoverySkipDMLAsStartTs := skipDMLAsStartTsList[0] - if recoveryStartTs > mergeStartTsCandidate { - finalStartTs = recoveryStartTs - finalSkipSyncpointAtStartTs = recoverySkipSyncpointAtStartTs - finalSkipDMLAsStartTs = recoverySkipDMLAsStartTs - } else if recoveryStartTs == mergeStartTsCandidate { - finalSkipSyncpointAtStartTs = mergeSkipSyncpointAtStartTsCandidate || recoverySkipSyncpointAtStartTs - finalSkipDMLAsStartTs = mergeSkipDMLAsStartTsCandidate || recoverySkipDMLAsStartTs - } - - log.Info("get table recovery info for merge dispatcher", - zap.Stringer("changefeedID", t.manager.changefeedID), - zap.Uint64("mergeStartTsCandidate", mergeStartTsCandidate), - zap.Any("recoveryStartTs", newStartTsList), - zap.Any("recoverySkipSyncpointAtStartTsList", skipSyncpointAtStartTsList), - zap.Any("recoverySkipDMLAsStartTsList", skipDMLAsStartTsList), - zap.Uint64("finalStartTs", finalStartTs), - zap.Bool("finalSkipSyncpointAtStartTs", finalSkipSyncpointAtStartTs), - zap.Bool("finalSkipDMLAsStartTs", finalSkipDMLAsStartTs), + zap.Uint64("pendingCommitTs", candidate.pendingCommitTs), + zap.Bool("pendingIsSyncPoint", candidate.pendingIsSyncPoint), + zap.Uint64("startTs", candidate.startTs), + zap.Bool("skipSyncpointAtStartTs", candidate.skipSyncpointAtStartTs), + zap.Bool("skipDMLAsStartTs", candidate.skipDMLAsStartTs), ) } - return finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs + return mergeMergedStartTsCandidateWithMySQLRecovery(t, candidate) } var ( diff --git a/maintainer/barrier_event.go b/maintainer/barrier_event.go index 7437d0995b..853772e7fc 100644 --- a/maintainer/barrier_event.go +++ b/maintainer/barrier_event.go @@ -458,7 +458,7 @@ func (be *BarrierEvent) sendPassAction(mode int64) []*messaging.TargetMessage { // check all related blocked dispatchers progress, to forward the progress of some block event, // to avoid the corner case that some dispatcher has forward checkpointTs. -// If the dispatcher's checkpointTs >= commitTs of this event, means the block event is writen to the sink. +// See forwardBarrierEvent for the exact forwarding rules. // // For example, there are two nodes A and B, and there are two dispatchers A1 and B1, maintainer is also running on A. // One ddl event E need the evolve of A1 and B1, and A1 finish flushing the event E downstream. @@ -527,7 +527,13 @@ func (be *BarrierEvent) checkBlockedDispatchers() { } } -// forwardBarrierEvent checks whether the barrier event can be forwarded for the given replication +// forwardBarrierEvent returns true if `replication` is known to have passed `event`. +// +// We intentionally avoid `checkpointTs >= commitTs`: a dispatcher may be recreated with +// `startTs == commitTs` and not skip the syncpoint at that ts, so it may report +// `checkpointTs == commitTs` before the syncpoint is actually flushed. We only forward when the +// replication is strictly beyond the barrier, or when ordering guarantees it (replication is in a +// syncpoint barrier at the same ts while `event` is a DDL barrier). func forwardBarrierEvent(replication *replica.SpanReplication, event *BarrierEvent) bool { if replication.GetStatus().CheckpointTs > event.commitTs { return true From 43c6b3f6d3443a8ba0c4ca88ac89db022c7e3679 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 12:54:37 +0800 Subject: [PATCH 15/20] update --- .../dispatcher/basic_dispatcher.go | 4 +- .../conf/changefeed.toml | 7 + .../conf/diff_config_part1.toml | 24 +++ .../conf/diff_config_part2.toml | 7 + .../run.sh | 168 ++++++++++++++++++ 5 files changed, 208 insertions(+), 2 deletions(-) create mode 100644 tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/changefeed.toml create mode 100644 tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part1.toml create mode 100644 tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part2.toml create mode 100644 tests/integration_tests/in_flight_syncpoint_during_scheduling/run.sh diff --git a/downstreamadapter/dispatcher/basic_dispatcher.go b/downstreamadapter/dispatcher/basic_dispatcher.go index afdf302ac5..a4b042225c 100644 --- a/downstreamadapter/dispatcher/basic_dispatcher.go +++ b/downstreamadapter/dispatcher/basic_dispatcher.go @@ -614,7 +614,7 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D actionCommitTs := action.CommitTs actionIsSyncPoint := action.IsSyncPoint d.sharedInfo.GetBlockEventExecutor().Submit(d, func() { - d.ExecutePendingDDL(pendingEvent, actionCommitTs, actionIsSyncPoint) + d.ExecuteBlockEventDDL(pendingEvent, actionCommitTs, actionIsSyncPoint) }) return true } else { @@ -648,7 +648,7 @@ func (d *BasicDispatcher) HandleDispatcherStatus(dispatcherStatus *heartbeatpb.D return false } -func (d *BasicDispatcher) ExecutePendingDDL(pendingEvent commonEvent.BlockEvent, actionCommitTs uint64, actionIsSyncPoint bool) { +func (d *BasicDispatcher) ExecuteBlockEventDDL(pendingEvent commonEvent.BlockEvent, actionCommitTs uint64, actionIsSyncPoint bool) { failpoint.Inject("BlockOrWaitBeforeWrite", nil) err := d.AddBlockEventToSink(pendingEvent) if err != nil { diff --git a/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/changefeed.toml b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/changefeed.toml new file mode 100644 index 0000000000..e83b0917c7 --- /dev/null +++ b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/changefeed.toml @@ -0,0 +1,7 @@ +enable-sync-point = true +sync-point-interval = "30s" + +[scheduler] +enable-table-across-nodes = true +region-threshold=1 +region-count-per-span=10 diff --git a/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part1.toml b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part1.toml new file mode 100644 index 0000000000..b1104b32f6 --- /dev/null +++ b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part1.toml @@ -0,0 +1,24 @@ +# diff Configuration. + +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] + output-dir = "/tmp/tidb_cdc_test/in_flight_syncpoint_during_scheduling/sync_diff/output" + + source-instances = ["tidb0"] + + target-instance = "mysql1" + + target-check-tables = ["sp_move.*"] + +[data-sources] +[data-sources.tidb0] + host = "127.0.0.1" + port = 4000 + user = "root" + password = "" + diff --git a/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part2.toml b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part2.toml new file mode 100644 index 0000000000..235ea5a8e3 --- /dev/null +++ b/tests/integration_tests/in_flight_syncpoint_during_scheduling/conf/diff_config_part2.toml @@ -0,0 +1,7 @@ + +[data-sources.mysql1] + host = "127.0.0.1" + port = 3306 + user = "root" + password = "" + diff --git a/tests/integration_tests/in_flight_syncpoint_during_scheduling/run.sh b/tests/integration_tests/in_flight_syncpoint_during_scheduling/run.sh new file mode 100644 index 0000000000..03db5c7c99 --- /dev/null +++ b/tests/integration_tests/in_flight_syncpoint_during_scheduling/run.sh @@ -0,0 +1,168 @@ +#!/bin/bash +# +# This test verifies that moving a dispatcher/merging a dispatcher during an in-flight syncpoint does not +# restart the dispatcher from (syncpoint_ts - 1). Otherwise, the dispatcher may re-scan and +# re-apply events with commitTs <= syncpoint_ts while the table-trigger dispatcher is writing +# the syncpoint, which can break the snapshot consistency semantics. + +set -eu + +CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $CUR/../_utils/test_prepare + +WORK_DIR=$OUT_DIR/$TEST_NAME +CDC_BINARY=cdc.test +SINK_TYPE=$1 + +DB_NAME="sp_move" +TABLE_NAME="t" +MERGE_TABLE_NAME="t_merge" +CHANGEFEED_ID="test" +dml_pid=0 + +deployConfig() { + cat $CUR/conf/diff_config_part1.toml >$CUR/conf/diff_config.toml + sed -i "s#output-dir = \"/tmp/tidb_cdc_test/in_flight_syncpoint_during_scheduling/sync_diff/output\"#output-dir = \"$WORK_DIR/sync_diff/output\"#g" $CUR/conf/diff_config.toml + echo "snapshot = \"$1\"" >>$CUR/conf/diff_config.toml + cat $CUR/conf/diff_config_part2.toml >>$CUR/conf/diff_config.toml + echo "snapshot = \"$2\"" >>$CUR/conf/diff_config.toml +} + +run() { + if [ "$SINK_TYPE" != "mysql" ]; then + echo "only mysql sink supports syncpoint record" + return + fi + + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" + + SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" + cat >$WORK_DIR/changefeed_scheduler.toml <= $syncpoint_ts" + exit 1 + fi + + # Restart node0 to disable BlockOrWaitBeforeWrite, so the pending syncpoint can be written downstream. + cdc_pid_0=$(get_cdc_pid "$CDC_HOST" "$CDC_PORT") + kill_cdc_pid $cdc_pid_0 + export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-2" --addr "127.0.0.1:8300" + + # Wait until the syncpoint is written downstream, then validate snapshot consistency by sync_diff. + ensure 120 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" + secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) + echo "secondary_ts: $secondary_ts" + + deployConfig "$syncpoint_ts" "$secondary_ts" + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 120 + rm -f $CUR/conf/diff_config.toml + + export GO_FAILPOINTS='' + cleanup_process $CDC_BINARY +} + +trap 'stop_test $WORK_DIR' EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 444c86bd53852e2352cbd9890c8074475b2ac3c4 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 12:55:56 +0800 Subject: [PATCH 16/20] update: --- downstreamadapter/dispatchermanager/task.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/downstreamadapter/dispatchermanager/task.go b/downstreamadapter/dispatchermanager/task.go index b5a8cb47e1..e0266fe47a 100644 --- a/downstreamadapter/dispatchermanager/task.go +++ b/downstreamadapter/dispatchermanager/task.go @@ -273,10 +273,10 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch // change the component status of the merged dispatcher to Initializing // set dispatcher into dispatcherMap and related field // notify eventCollector to update the merged dispatcher startTs - finalStartTs, finalSkipSyncpointAtStartTs, finalSkipDMLAsStartTs := resolveMergedDispatcherStartTs(t, minCheckpointTs, pendingStates) - t.mergedDispatcher.SetStartTs(finalStartTs) - t.mergedDispatcher.SetSkipSyncpointAtStartTs(finalSkipSyncpointAtStartTs) - t.mergedDispatcher.SetSkipDMLAsStartTs(finalSkipDMLAsStartTs) + startTs, skipSyncpointAtStartTs, skipDMLAsStartTs := resolveMergedDispatcherStartTs(t, minCheckpointTs, pendingStates) + t.mergedDispatcher.SetStartTs(startTs) + t.mergedDispatcher.SetSkipSyncpointAtStartTs(skipSyncpointAtStartTs) + t.mergedDispatcher.SetSkipDMLAsStartTs(skipDMLAsStartTs) t.mergedDispatcher.SetCurrentPDTs(t.manager.pdClock.CurrentTS()) t.mergedDispatcher.SetComponentStatus(heartbeatpb.ComponentState_Initializing) @@ -286,7 +286,7 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), zap.Int64("mode", t.mergedDispatcher.GetMode()), zap.Any("tableSpan", common.FormatTableSpan(t.mergedDispatcher.GetTableSpan())), - zap.Uint64("startTs", finalStartTs), + zap.Uint64("startTs", startTs), ) // Step3: cancel the merge task From 2bb4c17b54e0bf01a54f4569d268a83171a5bb53 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 12:57:11 +0800 Subject: [PATCH 17/20] update --- .../conf/changefeed.toml | 7 - .../conf/diff_config_part1.toml | 24 --- .../conf/diff_config_part2.toml | 7 - .../run.sh | 168 ------------------ 4 files changed, 206 deletions(-) delete mode 100644 tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml delete mode 100644 tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml delete mode 100644 tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml delete mode 100644 tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh diff --git a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml deleted file mode 100644 index e83b0917c7..0000000000 --- a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/changefeed.toml +++ /dev/null @@ -1,7 +0,0 @@ -enable-sync-point = true -sync-point-interval = "30s" - -[scheduler] -enable-table-across-nodes = true -region-threshold=1 -region-count-per-span=10 diff --git a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml deleted file mode 100644 index a464454bb0..0000000000 --- a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part1.toml +++ /dev/null @@ -1,24 +0,0 @@ -# diff Configuration. - -check-thread-count = 4 - -export-fix-sql = true - -check-struct-only = false - -[task] - output-dir = "/tmp/tidb_cdc_test/in_flight_syncpoint_during_sheduling/sync_diff/output" - - source-instances = ["tidb0"] - - target-instance = "mysql1" - - target-check-tables = ["sp_move.*"] - -[data-sources] -[data-sources.tidb0] - host = "127.0.0.1" - port = 4000 - user = "root" - password = "" - diff --git a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml b/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml deleted file mode 100644 index 235ea5a8e3..0000000000 --- a/tests/integration_tests/in_flight_syncpoint_during_sheduling/conf/diff_config_part2.toml +++ /dev/null @@ -1,7 +0,0 @@ - -[data-sources.mysql1] - host = "127.0.0.1" - port = 3306 - user = "root" - password = "" - diff --git a/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh b/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh deleted file mode 100644 index 5aff9d21b8..0000000000 --- a/tests/integration_tests/in_flight_syncpoint_during_sheduling/run.sh +++ /dev/null @@ -1,168 +0,0 @@ -#!/bin/bash -# -# This test verifies that moving a dispatcher/merging a dispatcher during an in-flight syncpoint does not -# restart the dispatcher from (syncpoint_ts - 1). Otherwise, the dispatcher may re-scan and -# re-apply events with commitTs <= syncpoint_ts while the table-trigger dispatcher is writing -# the syncpoint, which can break the snapshot consistency semantics. - -set -eu - -CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -source $CUR/../_utils/test_prepare - -WORK_DIR=$OUT_DIR/$TEST_NAME -CDC_BINARY=cdc.test -SINK_TYPE=$1 - -DB_NAME="sp_move" -TABLE_NAME="t" -MERGE_TABLE_NAME="t_merge" -CHANGEFEED_ID="test" -dml_pid=0 - -deployConfig() { - cat $CUR/conf/diff_config_part1.toml >$CUR/conf/diff_config.toml - sed -i "s#output-dir = \"/tmp/tidb_cdc_test/in_flight_syncpoint_during_sheduling/sync_diff/output\"#output-dir = \"$WORK_DIR/sync_diff/output\"#g" $CUR/conf/diff_config.toml - echo "snapshot = \"$1\"" >>$CUR/conf/diff_config.toml - cat $CUR/conf/diff_config_part2.toml >>$CUR/conf/diff_config.toml - echo "snapshot = \"$2\"" >>$CUR/conf/diff_config.toml -} - -run() { - if [ "$SINK_TYPE" != "mysql" ]; then - echo "only mysql sink supports syncpoint record" - return - fi - - rm -rf $WORK_DIR && mkdir -p $WORK_DIR - - start_tidb_cluster --workdir $WORK_DIR - - start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0" --addr "127.0.0.1:8300" - - SINK_URI="mysql://root@127.0.0.1:3306/?max-txn-row=1" - cat >$WORK_DIR/changefeed_scheduler.toml <= $syncpoint_ts" - exit 1 - fi - - # Restart node0 to disable BlockOrWaitBeforeWrite, so the pending syncpoint can be written downstream. - cdc_pid_0=$(get_cdc_pid "$CDC_HOST" "$CDC_PORT") - kill_cdc_pid $cdc_pid_0 - export GO_FAILPOINTS='github.com/pingcap/ticdc/maintainer/scheduler/StopBalanceScheduler=return(true)' - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "0-2" --addr "127.0.0.1:8300" - - # Wait until the syncpoint is written downstream, then validate snapshot consistency by sync_diff. - ensure 120 "mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e \"SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';\" | grep -E '^[0-9]+'" - secondary_ts=$(mysql -h${DOWN_TIDB_HOST} -P${DOWN_TIDB_PORT} -uroot -N -s -e "SELECT secondary_ts FROM tidb_cdc.syncpoint_v1 WHERE changefeed='default/${CHANGEFEED_ID}' AND primary_ts='${syncpoint_ts}';" | tail -n 1) - echo "secondary_ts: $secondary_ts" - - deployConfig "$syncpoint_ts" "$secondary_ts" - check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 120 - rm -f $CUR/conf/diff_config.toml - - export GO_FAILPOINTS='' - cleanup_process $CDC_BINARY -} - -trap 'stop_test $WORK_DIR' EXIT -run $* -check_logs $WORK_DIR -echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" From 47cc90382beacf772552935162ab5c812f4d6678 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 13:09:02 +0800 Subject: [PATCH 18/20] update --- downstreamadapter/dispatchermanager/task.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/downstreamadapter/dispatchermanager/task.go b/downstreamadapter/dispatchermanager/task.go index e0266fe47a..7d80baf64a 100644 --- a/downstreamadapter/dispatchermanager/task.go +++ b/downstreamadapter/dispatchermanager/task.go @@ -280,7 +280,7 @@ func doMerge[T dispatcher.Dispatcher](t *MergeCheckTask, dispatcherMap *Dispatch t.mergedDispatcher.SetCurrentPDTs(t.manager.pdClock.CurrentTS()) t.mergedDispatcher.SetComponentStatus(heartbeatpb.ComponentState_Initializing) - appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).CommitAddDispatcher(t.mergedDispatcher, finalStartTs) + appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).CommitAddDispatcher(t.mergedDispatcher, startTs) log.Info("merge dispatcher commit", zap.Stringer("changefeedID", t.manager.changefeedID), zap.Stringer("dispatcherID", t.mergedDispatcher.GetId()), From fe6934f860fc503a1aedf61961448be2585a86a7 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 25 Dec 2025 13:24:11 +0800 Subject: [PATCH 19/20] Update tests/integration_tests/run_heavy_it_in_ci.sh Co-authored-by: gemini-code-assist[bot] <176961590+gemini-code-assist[bot]@users.noreply.github.com> --- tests/integration_tests/run_heavy_it_in_ci.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index d4bce15cfe..8bb0d6a3f1 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -38,7 +38,7 @@ mysql_groups=( # G03 'cdc move_table in_flight_ddl_during_scheduling checkpoint_race_ddl_crash' # G04 - 'complex_transaction syncpoint in_flight_syncpoint_during_sheduling syncpoint_check_ts random_drop_message' + 'complex_transaction syncpoint in_flight_syncpoint_during_scheduling syncpoint_check_ts random_drop_message' # G05 'ddl_for_split_tables_with_merge_and_split' # G06 From 1bb1e157368c987d776ac8ab5664cbc3c5f24251 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 7 Jan 2026 14:07:49 +0800 Subject: [PATCH 20/20] update --- tests/integration_tests/run_heavy_it_in_ci.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration_tests/run_heavy_it_in_ci.sh b/tests/integration_tests/run_heavy_it_in_ci.sh index 50c2773758..226c2719e2 100755 --- a/tests/integration_tests/run_heavy_it_in_ci.sh +++ b/tests/integration_tests/run_heavy_it_in_ci.sh @@ -72,7 +72,8 @@ kafka_groups=( # G02 'canal_json_handle_key_only ddl_for_split_tables_with_failover' # G03 - 'canal_json_adapter_compatibility ddl_for_split_tables_with_merge_and_split' + # 'canal_json_adapter_compatibility ddl_for_split_tables_with_merge_and_split' + 'ddl_for_split_tables_with_merge_and_split' # G04 'open_protocol_claim_check open_protocol_handle_key_only random_drop_message' # G05