From 04d786799145053940da74f18df83336a67f7446 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 16 Apr 2025 08:30:48 +0000 Subject: [PATCH 01/13] init --- pkg/errors/error.go | 8 ++ pkg/sink/mysql/helper.go | 62 +++++++++++ pkg/sink/mysql/mysql_writer.go | 45 ++++---- pkg/sink/mysql/mysql_writer_ddl.go | 161 +++++++++-------------------- 4 files changed, 138 insertions(+), 138 deletions(-) diff --git a/pkg/errors/error.go b/pkg/errors/error.go index b59d9cb485..a73711d3e4 100644 --- a/pkg/errors/error.go +++ b/pkg/errors/error.go @@ -165,6 +165,14 @@ var ( "unknown '%s' message protocol for sink", errors.RFCCodeText("CDC:ErrSinkUnknownProtocol"), ) + ErrExecDDLFailed = errors.Normalize( + "exec DDL failed %s", + errors.RFCCodeText("CDC:ErrExecDDLFailed"), + ) + ErrDDLStateNotFound = errors.Normalize( + "DDL state not found %s", + errors.RFCCodeText("CDC:ErrDDLStateNotFound"), + ) ErrMySQLTxnError = errors.Normalize( "MySQL txn error", errors.RFCCodeText("CDC:ErrMySQLTxnError"), diff --git a/pkg/sink/mysql/helper.go b/pkg/sink/mysql/helper.go index ec7bf202b5..c4cacf693e 100644 --- a/pkg/sink/mysql/helper.go +++ b/pkg/sink/mysql/helper.go @@ -365,6 +365,17 @@ func needSwitchDB(event *commonEvent.DDLEvent) bool { return true } +func needWaitAsyncExecDone(t timodel.ActionType) bool { + switch t { + case timodel.ActionCreateTable, timodel.ActionCreateTables: + return false + case timodel.ActionCreateSchema: + return false + default: + return true + } +} + // SetWriteSource sets write source for the transaction. // When this variable is set to a value other than 0, data written in this session is considered to be written by TiCDC. // DDLs executed in a PRIMARY cluster can be replicated to a SECONDARY cluster by TiCDC. @@ -436,3 +447,54 @@ func getSQLErrCode(err error) (errors.ErrCode, bool) { return errors.ErrCode(mysqlErr.Number), true } + +func getDDLCreateTime(ctx context.Context, db *sql.DB) string { + ddlCreateTime := "" // default when scan failed + row, err := db.QueryContext(ctx, "BEGIN; SET @ticdc_ts := TIDB_PARSE_TSO(@@tidb_current_ts); ROLLBACK; SELECT @ticdc_ts; SET @ticdc_ts=NULL;") + if err != nil { + log.Warn("selecting tidb current timestamp failed", zap.Error(err)) + } else { + for row.Next() { + err = row.Scan(&ddlCreateTime) + if err != nil { + log.Warn("getting ddlCreateTime failed", zap.Error(err)) + } + } + //nolint:sqlclosecheck + _ = row.Close() + _ = row.Err() + } + return ddlCreateTime +} + +// getDDLStateFromTiDB retrieves the ddl job status of the ddl query from downstream tidb based on the ddl query and the approximate ddl create time. +func getDDLStateFromTiDB(ctx context.Context, db *sql.DB, ddl string, createTime string) (timodel.JobState, error) { + // ddlCreateTime and createTime are both based on UTC timezone of downstream + showJobs := fmt.Sprintf(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY FROM information_schema.ddl_jobs + WHERE CREATE_TIME >= "%s" AND QUERY = "%s";`, createTime, ddl) + //nolint:rowserrcheck + jobsRows, err := db.QueryContext(ctx, showJobs) + if err != nil { + return timodel.JobStateNone, err + } + + var jobsResults [][]string + jobsResults, err = export.GetSpecifiedColumnValuesAndClose(jobsRows, "QUERY", "STATE", "JOB_ID", "JOB_TYPE", "SCHEMA_STATE") + if err != nil { + return timodel.JobStateNone, err + } + if len(jobsResults) > 0 { + result := jobsResults[0] + state, jobID, jobType, schemaState := result[1], result[2], result[3], result[4] + log.Debug("Find ddl state in downstream", + zap.String("jobID", jobID), + zap.String("jobType", jobType), + zap.String("schemaState", schemaState), + zap.String("ddl", ddl), + zap.String("state", state), + zap.Any("jobsResults", jobsResults), + ) + return timodel.StrToJobState(result[1]), nil + } + return timodel.JobStateNone, nil +} diff --git a/pkg/sink/mysql/mysql_writer.go b/pkg/sink/mysql/mysql_writer.go index 34657829aa..f32f3dfda6 100644 --- a/pkg/sink/mysql/mysql_writer.go +++ b/pkg/sink/mysql/mysql_writer.go @@ -116,36 +116,27 @@ func (w *Writer) FlushDDLEvent(event *commonEvent.DDLEvent) error { w.waitAsyncDDLDone(event) } - // check the ddl should by async or sync executed. - if needAsyncExecDDL(event.GetDDLType()) && w.cfg.IsTiDB { - // for async exec ddl, we don't flush ddl ts here. Because they don't block checkpointTs. - err := w.asyncExecAddIndexDDLIfTimeout(event) - if err != nil { - return errors.Trace(err) - } - } else if !(event.TiDBOnly && !w.cfg.IsTiDB) { - if w.cfg.IsTiDB { - // if downstream is tidb, we write ddl ts before ddl first, and update the ddl ts item after ddl executed, - // to ensure the atomic with ddl writing when server is restarted. - w.FlushDDLTsPre(event) - } + if w.cfg.IsTiDB { + // if downstream is tidb, we write ddl ts before ddl first, and update the ddl ts item after ddl executed, + // to ensure the atomic with ddl writing when server is restarted. + w.FlushDDLTsPre(event) + } - err := w.execDDLWithMaxRetries(event) - if err != nil { - return err - } + err := w.execDDLWithMaxRetries(event) + if err != nil { + return err + } - // We need to record ddl' ts after each ddl for each table in the downstream when sink is mysql-compatible. - // Only in this way, when the node restart, we can continue sync data from the last ddl ts at least. - // Otherwise, after restarting, we may sync old data in new schema, which will leading to data loss. + // We need to record ddl' ts after each ddl for each table in the downstream when sink is mysql-compatible. + // Only in this way, when the node restart, we can continue sync data from the last ddl ts at least. + // Otherwise, after restarting, we may sync old data in new schema, which will leading to data loss. - // We make Flush ddl ts before callback(), in order to make sure the ddl ts is flushed - // before new checkpointTs will report to maintainer. Therefore, when the table checkpointTs is forward, - // we can ensure the ddl and ddl ts are both flushed downstream successfully. - err = w.FlushDDLTs(event) - if err != nil { - return err - } + // We make Flush ddl ts before callback(), in order to make sure the ddl ts is flushed + // before new checkpointTs will report to maintainer. Therefore, when the table checkpointTs is forward, + // we can ensure the ddl and ddl ts are both flushed downstream successfully. + err = w.FlushDDLTs(event) + if err != nil { + return err } for _, callback := range event.PostTxnFlushed { diff --git a/pkg/sink/mysql/mysql_writer_ddl.go b/pkg/sink/mysql/mysql_writer_ddl.go index 6ff1f37bb8..82f9e67805 100644 --- a/pkg/sink/mysql/mysql_writer_ddl.go +++ b/pkg/sink/mysql/mysql_writer_ddl.go @@ -18,103 +18,18 @@ import ( "fmt" "time" - "github.com/pingcap/errors" + "github.com/go-sql-driver/mysql" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/apperror" "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/retry" timodel "github.com/pingcap/tidb/pkg/meta/model" pmysql "github.com/pingcap/tiflow/pkg/sink/mysql" "go.uber.org/zap" ) -func (w *Writer) asyncExecAddIndexDDLIfTimeout(event *commonEvent.DDLEvent) error { - var tableIDs []int64 - switch event.GetBlockedTables().InfluenceType { - // only normal type may have ddl need to async exec - case commonEvent.InfluenceTypeNormal: - tableIDs = event.GetBlockedTables().TableIDs - default: - } - - for _, tableID := range tableIDs { - // change the async ddl state to 1, means the tableID have async table ddl executing - w.asyncDDLState.Store(tableID, 1) - } - - done := make(chan error, 1) - - tick := time.NewTimer(2 * time.Second) - defer tick.Stop() - log.Info("async exec add index ddl start", - zap.Uint64("commitTs", event.FinishedTs), - zap.String("ddl", event.GetDDLQuery())) - go func() { - if err := w.execDDLWithMaxRetries(event); err != nil { - log.Error("async exec add index ddl failed", - zap.Uint64("commitTs", event.FinishedTs), - zap.String("ddl", event.GetDDLQuery())) - done <- err - return - } - log.Info("async exec add index ddl done", - zap.Uint64("commitTs", event.FinishedTs), - zap.String("ddl", event.GetDDLQuery())) - done <- nil - - for _, tableID := range tableIDs { - // change the async ddl state to 2, means the tableID don't have async table ddl executing - w.asyncDDLState.Store(tableID, 2) - } - }() - - select { - case err := <-done: - // if the ddl is executed within 2 seconds, we just return the result to the caller. - return err - case <-tick.C: - // if the ddl is still running, we just return nil, - // then if the ddl is failed, the downstream ddl is lost. - // because the checkpoint ts is forwarded. - log.Info("async add index ddl is still running", - zap.Uint64("commitTs", event.FinishedTs), - zap.String("ddl", event.GetDDLQuery())) - return nil - } -} - -func needAsyncExecDDL(ddlType timodel.ActionType) bool { - switch ddlType { - case timodel.ActionAddIndex: - return true - default: - return false - } -} - -func needTimeoutCheck(ddlType timodel.ActionType) bool { - if needAsyncExecDDL(ddlType) { - return false - } - switch ddlType { - // partition related - case timodel.ActionAddTablePartition, timodel.ActionExchangeTablePartition, timodel.ActionReorganizePartition: - return false - // reorg related - case timodel.ActionAddPrimaryKey, timodel.ActionAddIndex, timodel.ActionModifyColumn: - return false - // following ddls can be fast when the downstream is TiDB, we must - // still take them into consideration to ensure compatibility with all - // MySQL-compatible databases. - case timodel.ActionAddColumn, timodel.ActionAddColumns, timodel.ActionDropColumn, timodel.ActionDropColumns: - return false - default: - return true - } -} - func (w *Writer) execDDL(event *commonEvent.DDLEvent) error { if w.cfg.DryRun { log.Info("Dry run DDL", zap.String("sql", event.GetDDLQuery())) @@ -137,15 +52,6 @@ func (w *Writer) execDDL(event *commonEvent.DDLEvent) error { } ctx := w.ctx - // We check the most of the ddl event executed with timeout. - if needTimeoutCheck(event.GetDDLType()) { - writeTimeout, _ := time.ParseDuration(w.cfg.WriteTimeout) - writeTimeout += networkDriftDuration - var cancelFunc func() - ctx, cancelFunc = context.WithTimeout(w.ctx, writeTimeout) - defer cancelFunc() - } - shouldSwitchDB := needSwitchDB(event) // Convert vector type to string type for unsupport database @@ -192,13 +98,14 @@ func (w *Writer) execDDL(event *commonEvent.DDLEvent) error { } if err = tx.Commit(); err != nil { - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Query info: %s; ", event.GetDDLQuery()))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Query info: %s; ", event.GetDDLQuery()))) } return nil } func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { + ddlCreateTime := getDDLCreateTime(w.ctx, w.db) return retry.Do(w.ctx, func() error { err := w.statistics.RecordDDLExecution(func() error { return w.execDDL(event) }) if err != nil { @@ -210,10 +117,15 @@ func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { // If the error is ignorable, we will ignore the error directly. return nil } + if w.cfg.IsTiDB && ddlCreateTime != "" && errors.Cause(err) == mysql.ErrInvalidConn { + log.Warn("Wait the asynchronous ddl to synchronize", zap.String("ddl", event.Query), zap.String("ddlCreateTime", ddlCreateTime), + zap.String("readTimeout", w.cfg.ReadTimeout), zap.Error(err)) + return w.waitDDLDone(w.ctx, event, ddlCreateTime) + } log.Warn("Execute DDL with error, retry later", zap.String("ddl", event.Query), zap.Error(err)) - return cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Execute DDL failed, Query info: %s; ", event.GetDDLQuery()))) + return errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("Execute DDL failed, Query info: %s; ", event.GetDDLQuery()))) } log.Info("Execute DDL succeeded", zap.String("changefeed", w.ChangefeedID.String()), @@ -225,6 +137,44 @@ func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { retry.WithIsRetryableErr(apperror.IsRetryableDDLError)) } +func (w *Writer) waitDDLDone(ctx context.Context, ddl *commonEvent.DDLEvent, ddlCreateTime string) error { + ticker := time.NewTicker(5 * time.Second) + ticker1 := time.NewTicker(10 * time.Minute) + defer ticker.Stop() + defer ticker1.Stop() + for { + state, err := getDDLStateFromTiDB(ctx, w.db, ddl.Query, ddlCreateTime) + if err != nil { + log.Error("Error when getting DDL state from TiDB", zap.Error(err)) + } + switch state { + case timodel.JobStateDone, timodel.JobStateSynced: + log.Info("DDL replicate success", zap.String("ddl", ddl.Query), zap.String("ddlCreateTime", ddlCreateTime)) + return nil + case timodel.JobStateCancelled, timodel.JobStateRollingback, timodel.JobStateRollbackDone, timodel.JobStateCancelling: + return errors.ErrExecDDLFailed.GenWithStackByArgs(ddl.Query) + case timodel.JobStateRunning, timodel.JobStateQueueing: + switch ddl.GetDDLType() { + // returned immediately if not block dml + case timodel.ActionAddIndex: + log.Info("DDL is running downstream", zap.String("ddl", ddl.Query), zap.String("ddlCreateTime", ddlCreateTime), zap.Any("ddlState", state)) + return nil + } + default: + log.Warn("Unexpected DDL state, may not be found downstream", zap.String("ddl", ddl.Query), zap.String("ddlCreateTime", ddlCreateTime), zap.Any("ddlState", state)) + return errors.ErrDDLStateNotFound.GenWithStackByArgs(state) + } + + select { + case <-ctx.Done(): + return ctx.Err() + case <-ticker.C: + case <-ticker1.C: + log.Info("DDL is still running downstream, it blocks other DDL or DML events", zap.String("ddl", ddl.Query), zap.String("ddlCreateTime", ddlCreateTime)) + } + } +} + func (w *Writer) waitAsyncDDLDone(event *commonEvent.DDLEvent) { if !needWaitAsyncExecDone(event.GetDDLType()) { return @@ -288,7 +238,7 @@ func (w *Writer) doQueryAsyncDDL(tableID int64, query string) (bool, error) { rows, err := w.db.QueryContext(w.ctx, query) log.Debug("query duration", zap.Any("duration", time.Since(start)), zap.Any("query", query)) if err != nil { - return false, cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("failed to query ddl jobs table; Query is %s", query))) + return false, errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("failed to query ddl jobs table; Query is %s", query))) } defer rows.Close() @@ -302,7 +252,7 @@ func (w *Writer) doQueryAsyncDDL(tableID int64, query string) (bool, error) { noRows = false err := rows.Scan(&jobID, &jobType, &schemaState, &state) if err != nil { - return false, cerror.WrapError(cerror.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("failed to query ddl jobs table; Query is %s", query))) + return false, errors.WrapError(errors.ErrMySQLTxnError, errors.WithMessage(err, fmt.Sprintf("failed to query ddl jobs table; Query is %s", query))) } log.Info("async ddl is still running", @@ -353,14 +303,3 @@ func (w *Writer) checkAndWaitAsyncDDLDoneDownstream(tableID int64) error { } } } - -func needWaitAsyncExecDone(t timodel.ActionType) bool { - switch t { - case timodel.ActionCreateTable, timodel.ActionCreateTables: - return false - case timodel.ActionCreateSchema: - return false - default: - return true - } -} From 7046f7e995ed7f2df0526863f461358879b68c20 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 16 Apr 2025 08:37:07 +0000 Subject: [PATCH 02/13] add test --- .../ddl_wait/conf/diff_config.toml | 28 ++++++ tests/integration_tests/ddl_wait/run.sh | 46 +++++++++ tests/integration_tests/ddl_wait/test.go | 97 +++++++++++++++++++ tests/integration_tests/run_light_it_in_ci.sh | 2 +- 4 files changed, 172 insertions(+), 1 deletion(-) create mode 100644 tests/integration_tests/ddl_wait/conf/diff_config.toml create mode 100755 tests/integration_tests/ddl_wait/run.sh create mode 100644 tests/integration_tests/ddl_wait/test.go diff --git a/tests/integration_tests/ddl_wait/conf/diff_config.toml b/tests/integration_tests/ddl_wait/conf/diff_config.toml new file mode 100644 index 0000000000..6435d3b89b --- /dev/null +++ b/tests/integration_tests/ddl_wait/conf/diff_config.toml @@ -0,0 +1,28 @@ +# diff Configuration. +check-thread-count = 4 + +export-fix-sql = true + +check-struct-only = false + +[task] +output-dir = "/tmp/tidb_cdc_test/ddl_wait/sync_diff/output" + +source-instances = ["mysql1"] + +target-instance = "tidb0" + +target-check-tables = ["test.*"] + +[data-sources] +[data-sources.mysql1] +host = "127.0.0.1" +port = 4000 +user = "root" +password = "" + +[data-sources.tidb0] +host = "127.0.0.1" +port = 3306 +user = "root" +password = "" diff --git a/tests/integration_tests/ddl_wait/run.sh b/tests/integration_tests/ddl_wait/run.sh new file mode 100755 index 0000000000..e2534dc499 --- /dev/null +++ b/tests/integration_tests/ddl_wait/run.sh @@ -0,0 +1,46 @@ +#!/bin/bash + +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 + +# This test simulates DDL operations that take a long time. +# TiCDC blocks DDL operations until its state is not running, except for adding indexes. +# TiCDC also checks add index ddl state before execute a new DDL. +function run() { + rm -rf $WORK_DIR && mkdir -p $WORK_DIR + + start_tidb_cluster --workdir $WORK_DIR + + cd $WORK_DIR + + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY + + cd $CUR + GO111MODULE=on go run test.go + + TOPIC_NAME="ticdc-ddl-wait-test-$RANDOM" + SINK_URI="mysql://root@127.0.0.1:3306/?read-timeout=30ms" + + changefeed_id="ddl-wait" + run_cdc_cli changefeed create --sink-uri="$SINK_URI" -c=${changefeed_id} + + run_sql "alter table test.t modify column col decimal(30,10);" + run_sql "alter table test.t add index (col);" + run_sql "create table test.finish_mark (a int primary key);" + check_table_exists test.finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 300 + # make sure all tables are equal in upstream and downstream + check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml 180 + check_logs_contains $WORK_DIR "DDL replicate success" + check_logs_contains $WORK_DIR "DDL is running downstream" + cleanup_process $CDC_BINARY +} + +trap stop_tidb_cluster EXIT +run $* +check_logs $WORK_DIR +echo "[$(date)] <<<<<< run test case $TEST_NAME success! >>>>>>" diff --git a/tests/integration_tests/ddl_wait/test.go b/tests/integration_tests/ddl_wait/test.go new file mode 100644 index 0000000000..204e385a03 --- /dev/null +++ b/tests/integration_tests/ddl_wait/test.go @@ -0,0 +1,97 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package main + +import ( + "database/sql" + "fmt" + "log" + "os" + "sync" + + _ "github.com/go-sql-driver/mysql" +) + +func main() { + upHost := GetEnvDefault("UP_TIDB_HOST", "127.0.0.1") + upPort := GetEnvDefault("UP_TIDB_PORT", "4000") + downHost := GetEnvDefault("UP_TIDB_HOST", "127.0.0.1") + downPort := GetEnvDefault("UP_TIDB_PORT", "3306") + + var wg sync.WaitGroup + wg.Add(2) + go func() { + defer wg.Done() + run(upHost, upPort) + }() + go func() { + defer wg.Done() + run(downHost, downPort) + }() + wg.Wait() +} + +func run(host string, port string) { + dsn := fmt.Sprintf("root@tcp(%s:%s)/", host, port) + db, err := sql.Open("mysql", dsn) + if err != nil { + log.Fatal("open db failed:", dsn, ", err: ", err) + } + defer db.Close() + + if err = db.Ping(); err != nil { + log.Fatal("ping db failed:", dsn, ", err: ", err) + } + log.Println("connect to tidb success, dsn: ", dsn) + + createTable := `create table if not exists test.t ( + id int primary key, + col decimal(65,30) + );` + insertDML := `insert into test.t values (?, ?)` + + concurrency := 200 + maxRowCnt := 1000000 + num := maxRowCnt / concurrency + db.SetMaxOpenConns(concurrency) + + _, err = db.Exec(createTable) + if err != nil { + log.Fatal("create table failed:, err: ", err) + } + + var wg sync.WaitGroup + for k := 0; k < concurrency; k++ { + wg.Add(1) + go func(k int) { + defer wg.Done() + for i := 0; i < num; i++ { + val := k*num + i + _, err = db.Exec(insertDML, val, val) + if err != nil { + log.Fatal("insert value failed:, err: ", err) + } + } + }(k) + } + wg.Wait() +} + +func GetEnvDefault(key, defaultV string) string { + val, ok := os.LookupEnv(key) + if !ok { + return defaultV + } + return val +} diff --git a/tests/integration_tests/run_light_it_in_ci.sh b/tests/integration_tests/run_light_it_in_ci.sh index 7af96f55f4..2bba758c4e 100755 --- a/tests/integration_tests/run_light_it_in_ci.sh +++ b/tests/integration_tests/run_light_it_in_ci.sh @@ -54,7 +54,7 @@ mysql_groups=( # G10 'changefeed_error bdr_mode fail_over_ddl_K' # G11 - 'multi_tables_ddl ddl_attributes multi_cdc_cluster fail_over_ddl_L' + 'multi_tables_ddl ddl_attributes multi_cdc_cluster ddl_wait fail_over_ddl_L' # G12 'row_format tiflash multi_rocks fail_over_ddl_M' # G13 From 2febb2bd84a59d8fe48fe76b30dd235cbd27b388 Mon Sep 17 00:00:00 2001 From: wk989898 Date: Wed, 16 Apr 2025 10:01:44 +0000 Subject: [PATCH 03/13] update --- pkg/sink/mysql/mysql_writer.go | 42 ++++++++++++++--------------- pkg/sink/mysql/mysql_writer_test.go | 1 + 2 files changed, 22 insertions(+), 21 deletions(-) diff --git a/pkg/sink/mysql/mysql_writer.go b/pkg/sink/mysql/mysql_writer.go index f32f3dfda6..ccd439dc68 100644 --- a/pkg/sink/mysql/mysql_writer.go +++ b/pkg/sink/mysql/mysql_writer.go @@ -115,28 +115,28 @@ func (w *Writer) FlushDDLEvent(event *commonEvent.DDLEvent) error { // first we check whether there is some async ddl executed now. w.waitAsyncDDLDone(event) } + if !(event.TiDBOnly && !w.cfg.IsTiDB) { + if w.cfg.IsTiDB { + // if downstream is tidb, we write ddl ts before ddl first, and update the ddl ts item after ddl executed, + // to ensure the atomic with ddl writing when server is restarted. + w.FlushDDLTsPre(event) + } - if w.cfg.IsTiDB { - // if downstream is tidb, we write ddl ts before ddl first, and update the ddl ts item after ddl executed, - // to ensure the atomic with ddl writing when server is restarted. - w.FlushDDLTsPre(event) - } - - err := w.execDDLWithMaxRetries(event) - if err != nil { - return err - } - - // We need to record ddl' ts after each ddl for each table in the downstream when sink is mysql-compatible. - // Only in this way, when the node restart, we can continue sync data from the last ddl ts at least. - // Otherwise, after restarting, we may sync old data in new schema, which will leading to data loss. - - // We make Flush ddl ts before callback(), in order to make sure the ddl ts is flushed - // before new checkpointTs will report to maintainer. Therefore, when the table checkpointTs is forward, - // we can ensure the ddl and ddl ts are both flushed downstream successfully. - err = w.FlushDDLTs(event) - if err != nil { - return err + err := w.execDDLWithMaxRetries(event) + if err != nil { + return err + } + // We need to record ddl' ts after each ddl for each table in the downstream when sink is mysql-compatible. + // Only in this way, when the node restart, we can continue sync data from the last ddl ts at least. + // Otherwise, after restarting, we may sync old data in new schema, which will leading to data loss. + + // We make Flush ddl ts before callback(), in order to make sure the ddl ts is flushed + // before new checkpointTs will report to maintainer. Therefore, when the table checkpointTs is forward, + // we can ensure the ddl and ddl ts are both flushed downstream successfully. + err = w.FlushDDLTs(event) + if err != nil { + return err + } } for _, callback := range event.PostTxnFlushed { diff --git a/pkg/sink/mysql/mysql_writer_test.go b/pkg/sink/mysql/mysql_writer_test.go index 686f35affb..8f0829bad4 100644 --- a/pkg/sink/mysql/mysql_writer_test.go +++ b/pkg/sink/mysql/mysql_writer_test.go @@ -375,6 +375,7 @@ func TestMysqlWriter_AsyncDDL(t *testing.T) { Query: job.Query, SchemaName: job.SchemaName, TableName: job.TableName, + Type: byte(job.Type), FinishedTs: 1, BlockedTables: &commonEvent.InfluencedTables{ InfluenceType: commonEvent.InfluenceTypeNormal, From e326ebe49abfa7573940fa6f34acb872500060bf Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 17 Apr 2025 06:27:36 +0000 Subject: [PATCH 04/13] fix --- pkg/sink/mysql/helper.go | 13 ++++++-- pkg/sink/mysql/mysql_writer.go | 6 ---- pkg/sink/mysql/mysql_writer_ddl.go | 48 +++++++++-------------------- pkg/sink/mysql/mysql_writer_test.go | 27 ++++++++++------ 4 files changed, 43 insertions(+), 51 deletions(-) diff --git a/pkg/sink/mysql/helper.go b/pkg/sink/mysql/helper.go index c4cacf693e..61daf319c5 100644 --- a/pkg/sink/mysql/helper.go +++ b/pkg/sink/mysql/helper.go @@ -37,6 +37,16 @@ import ( "go.uber.org/zap" ) +const checkRunningAddIndexSQL = ` +SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY +FROM information_schema.ddl_jobs +WHERE TABLE_ID = "%d" + AND JOB_TYPE LIKE "add index%%" + AND (STATE = "running" OR STATE = "queueing"); +` +const checkRunningSQL = `SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY FROM information_schema.ddl_jobs + WHERE CREATE_TIME >= "%s" AND QUERY = "%s";` + // CheckIfBDRModeIsSupported checks if the downstream supports BDR mode. func CheckIfBDRModeIsSupported(ctx context.Context, db *sql.DB) (bool, error) { isTiDB := CheckIsTiDB(ctx, db) @@ -470,8 +480,7 @@ func getDDLCreateTime(ctx context.Context, db *sql.DB) string { // getDDLStateFromTiDB retrieves the ddl job status of the ddl query from downstream tidb based on the ddl query and the approximate ddl create time. func getDDLStateFromTiDB(ctx context.Context, db *sql.DB, ddl string, createTime string) (timodel.JobState, error) { // ddlCreateTime and createTime are both based on UTC timezone of downstream - showJobs := fmt.Sprintf(`SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY FROM information_schema.ddl_jobs - WHERE CREATE_TIME >= "%s" AND QUERY = "%s";`, createTime, ddl) + showJobs := fmt.Sprintf(checkRunningSQL, createTime, ddl) //nolint:rowserrcheck jobsRows, err := db.QueryContext(ctx, showJobs) if err != nil { diff --git a/pkg/sink/mysql/mysql_writer.go b/pkg/sink/mysql/mysql_writer.go index ccd439dc68..cdaf8a92f6 100644 --- a/pkg/sink/mysql/mysql_writer.go +++ b/pkg/sink/mysql/mysql_writer.go @@ -16,7 +16,6 @@ package mysql import ( "context" "database/sql" - "sync" "time" lru "github.com/hashicorp/golang-lru" @@ -51,10 +50,6 @@ type Writer struct { ddlTsTableInit bool tableSchemaStore *util.TableSchemaStore - // asyncDDLState is used to store the state of async ddl. - // key: tableID, value: state(0: unknown state , 1: executing, 2: no executing ddl) - asyncDDLState sync.Map - // implement stmtCache to improve performance, especially when the downstream is TiDB stmtCache *lru.Cache // Indicate if the CachePrepStmts should be enabled or not @@ -84,7 +79,6 @@ func NewWriter( ChangefeedID: changefeedID, lastCleanSyncPointTime: time.Now(), ddlTsTableInit: false, - asyncDDLState: sync.Map{}, cachePrepStmts: cfg.CachePrepStmts, maxAllowedPacket: cfg.MaxAllowedPacket, stmtCache: cfg.stmtCache, diff --git a/pkg/sink/mysql/mysql_writer_ddl.go b/pkg/sink/mysql/mysql_writer_ddl.go index 82f9e67805..79b017d126 100644 --- a/pkg/sink/mysql/mysql_writer_ddl.go +++ b/pkg/sink/mysql/mysql_writer_ddl.go @@ -104,6 +104,10 @@ func (w *Writer) execDDL(event *commonEvent.DDLEvent) error { return nil } +// execDDLWithMaxRetries will retry executing DDL statements. +// When a DDL execution takes a long time and an invalid connection error occurs. +// If the downstream is TiDB, it will query the DDL and wait until it finishes. +// For 'add index' ddl, it will return immediately without waiting and will query it during the next DDL execution. func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { ddlCreateTime := getDDLCreateTime(w.ctx, w.db) return retry.Do(w.ctx, func() error { @@ -137,6 +141,7 @@ func (w *Writer) execDDLWithMaxRetries(event *commonEvent.DDLEvent) error { retry.WithIsRetryableErr(apperror.IsRetryableDDLError)) } +// waitDDLDone wait current ddl func (w *Writer) waitDDLDone(ctx context.Context, ddl *commonEvent.DDLEvent, ddlCreateTime string) error { ticker := time.NewTicker(5 * time.Second) ticker1 := time.NewTicker(10 * time.Minute) @@ -175,6 +180,7 @@ func (w *Writer) waitDDLDone(ctx context.Context, ddl *commonEvent.DDLEvent, ddl } } +// waitAsyncDDLDone wait previous ddl func (w *Writer) waitAsyncDDLDone(event *commonEvent.DDLEvent) { if !needWaitAsyncExecDone(event.GetDDLType()) { return @@ -194,44 +200,18 @@ func (w *Writer) waitAsyncDDLDone(event *commonEvent.DDLEvent) { if tableID == 0 { continue } - state, ok := w.asyncDDLState.Load(tableID) - if !ok { - // query the downstream, - // if the ddl is still running, we should wait for it. - w.checkAndWaitAsyncDDLDoneDownstream(tableID) - // update async ddl state - w.asyncDDLState.Store(tableID, 2) - // TODO - } else if state.(int) == 1 { - w.waitTableAsyncDDLDone(tableID) - } - } -} - -func (w *Writer) waitTableAsyncDDLDone(tableID int64) { - ticker := time.NewTicker(5 * time.Second) - defer ticker.Stop() - for { - select { - case <-w.ctx.Done(): - return - case <-ticker.C: - state, ok := w.asyncDDLState.Load(tableID) - if ok && state.(int) == 2 { - return - } + // query the downstream, + // if the ddl is still running, we should wait for it. + err := w.checkAndWaitAsyncDDLDoneDownstream(tableID) + if err != nil { + log.Error("check previous asynchronous ddl failed", + zap.String("namespace", w.ChangefeedID.Namespace()), + zap.Stringer("changefeed", w.ChangefeedID), + zap.Error(err)) } } } -var checkRunningAddIndexSQL = ` -SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY -FROM information_schema.ddl_jobs -WHERE TABLE_ID = "%d" - AND JOB_TYPE LIKE "add index%%" - AND (STATE = "running" OR STATE = "queueing"); -` - // true means the async ddl is still running, false means the async ddl is done. func (w *Writer) doQueryAsyncDDL(tableID int64, query string) (bool, error) { start := time.Now() diff --git a/pkg/sink/mysql/mysql_writer_test.go b/pkg/sink/mysql/mysql_writer_test.go index 8f0829bad4..54d74bbec9 100644 --- a/pkg/sink/mysql/mysql_writer_test.go +++ b/pkg/sink/mysql/mysql_writer_test.go @@ -16,10 +16,12 @@ package mysql import ( "context" "database/sql" + "fmt" "testing" "time" "github.com/DATA-DOG/go-sqlmock" + "github.com/go-sql-driver/mysql" "github.com/pingcap/log" "github.com/pingcap/ticdc/heartbeatpb" "github.com/pingcap/ticdc/pkg/common" @@ -407,11 +409,11 @@ func TestMysqlWriter_AsyncDDL(t *testing.T) { mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '1', 0, '', '', 0, 0), ('default', 'test/test', '1', 1, '', '', 0, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() + mock.ExpectQuery("BEGIN; SET @ticdc_ts := TIDB_PARSE_TSO(@@tidb_current_ts); ROLLBACK; SELECT @ticdc_ts; SET @ticdc_ts=NULL;").WillReturnRows(sqlmock.NewRows([]string{"@ticdc_ts"}).AddRow("2021-05-26 11:33:37.776000")) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectExec("create table t (id int primary key, name varchar(32));").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - mock.ExpectBegin() mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '1', 0, '', '', 1, 0), ('default', 'test/test', '1', 1, '', '', 1, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() @@ -423,11 +425,22 @@ func TestMysqlWriter_AsyncDDL(t *testing.T) { err = mock.ExpectationsWereMet() require.NoError(t, err) + mock.ExpectQuery(fmt.Sprintf(checkRunningAddIndexSQL, 1)).WillReturnError(sqlmock.ErrCancelled) + mock.ExpectBegin() + mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '1', 1, '', '', 0, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + mock.ExpectQuery("BEGIN; SET @ticdc_ts := TIDB_PARSE_TSO(@@tidb_current_ts); ROLLBACK; SELECT @ticdc_ts; SET @ticdc_ts=NULL;").WillReturnRows(sqlmock.NewRows([]string{"@ticdc_ts"}).AddRow("2021-05-26 11:33:37.776000")) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) log.Info("before add index") - mock.ExpectExec("alter table t add index nameIndex(name);").WillDelayFor(10 * time.Second).WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectExec("alter table t add index nameIndex(name);").WillDelayFor(10 * time.Second).WillReturnError(mysql.ErrInvalidConn) log.Info("after add index") + mock.ExpectQuery(fmt.Sprintf(checkRunningSQL, "2021-05-26 11:33:37.776000", "alter table t add index nameIndex(name);")). + WillReturnRows(sqlmock.NewRows([]string{"JOB_ID", "JOB_TYPE", "SCHEMA_STATE", "SCHEMA_ID", "TABLE_ID", "STATE", "QUERY"}). + AddRow("", "", "", "", "", "running", "alter table t add index nameIndex(name);")) + mock.ExpectBegin() + mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '1', 1, '', '', 1, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() // for dml event mock.ExpectBegin() @@ -440,29 +453,25 @@ func TestMysqlWriter_AsyncDDL(t *testing.T) { mock.ExpectBegin() mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '2', 0, '', '', 0, 0), ('default', 'test/test', '2', 2, '', '', 0, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - + mock.ExpectQuery("BEGIN; SET @ticdc_ts := TIDB_PARSE_TSO(@@tidb_current_ts); ROLLBACK; SELECT @ticdc_ts; SET @ticdc_ts=NULL;").WillReturnRows(sqlmock.NewRows([]string{"@ticdc_ts"}).AddRow("2021-05-26 11:33:37.776000")) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectExec("create table t1 (id int primary key, name varchar(32));").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - mock.ExpectBegin() mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '2', 0, '', '', 1, 0), ('default', 'test/test', '2', 2, '', '', 1, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - // this commit is for add index ddl - mock.ExpectCommit() - // for add column ddl for table t + mock.ExpectQuery(fmt.Sprintf(checkRunningAddIndexSQL, 1)).WillReturnError(sqlmock.ErrCancelled) mock.ExpectBegin() mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '3', 1, '', '', 0, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - + mock.ExpectQuery("BEGIN; SET @ticdc_ts := TIDB_PARSE_TSO(@@tidb_current_ts); ROLLBACK; SELECT @ticdc_ts; SET @ticdc_ts=NULL;").WillReturnRows(sqlmock.NewRows([]string{"@ticdc_ts"}).AddRow("2021-05-26 11:33:37.776000")) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectExec("alter table t add column age int;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() - mock.ExpectBegin() mock.ExpectExec("INSERT INTO tidb_cdc.ddl_ts_v1 (ticdc_cluster_id, changefeed, ddl_ts, table_id, table_name_in_ddl_job, db_name_in_ddl_job, finished, is_syncpoint) VALUES ('default', 'test/test', '3', 1, '', '', 1, 0) ON DUPLICATE KEY UPDATE finished=VALUES(finished), table_name_in_ddl_job=VALUES(table_name_in_ddl_job), db_name_in_ddl_job=VALUES(db_name_in_ddl_job), ddl_ts=VALUES(ddl_ts), created_at=NOW(), is_syncpoint=VALUES(is_syncpoint);").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit() From 0caf54052ffb9bb0d0ed0f43585bed24ea83a73f Mon Sep 17 00:00:00 2001 From: wk989898 Date: Thu, 17 Apr 2025 08:08:09 +0000 Subject: [PATCH 05/13] fix --- pkg/sink/mysql/helper.go | 1 + tests/integration_tests/ddl_wait/run.sh | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/sink/mysql/helper.go b/pkg/sink/mysql/helper.go index 61daf319c5..415df2ff4f 100644 --- a/pkg/sink/mysql/helper.go +++ b/pkg/sink/mysql/helper.go @@ -44,6 +44,7 @@ WHERE TABLE_ID = "%d" AND JOB_TYPE LIKE "add index%%" AND (STATE = "running" OR STATE = "queueing"); ` + const checkRunningSQL = `SELECT JOB_ID, JOB_TYPE, SCHEMA_STATE, SCHEMA_ID, TABLE_ID, STATE, QUERY FROM information_schema.ddl_jobs WHERE CREATE_TIME >= "%s" AND QUERY = "%s";` diff --git a/tests/integration_tests/ddl_wait/run.sh b/tests/integration_tests/ddl_wait/run.sh index e2534dc499..f9b43e4813 100755 --- a/tests/integration_tests/ddl_wait/run.sh +++ b/tests/integration_tests/ddl_wait/run.sh @@ -24,7 +24,7 @@ function run() { GO111MODULE=on go run test.go TOPIC_NAME="ticdc-ddl-wait-test-$RANDOM" - SINK_URI="mysql://root@127.0.0.1:3306/?read-timeout=30ms" + SINK_URI="mysql://root@127.0.0.1:3306/?read-timeout=300ms" changefeed_id="ddl-wait" run_cdc_cli changefeed create --sink-uri="$SINK_URI" -c=${changefeed_id} From f6797ecae859814b1b7c799b7d7400860decff90 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Wed, 16 Apr 2025 15:53:30 +0800 Subject: [PATCH 06/13] codec: fix the open protocol flag (#1228) close pingcap/ticdc#1227 --- pkg/sink/codec/open/codec.go | 72 ++++++++-------- pkg/sink/codec/open/decoder.go | 16 ++-- pkg/sink/codec/open/encoder.go | 24 ++++-- pkg/sink/codec/open/encoder_test.go | 113 +++++++++++++++++++------ pkg/sink/codec/open/message.go | 127 ++++++++++++++++++++++++++-- 5 files changed, 273 insertions(+), 79 deletions(-) diff --git a/pkg/sink/codec/open/codec.go b/pkg/sink/codec/open/codec.go index 7c5c8816f0..31e93056d5 100644 --- a/pkg/sink/codec/open/codec.go +++ b/pkg/sink/codec/open/codec.go @@ -30,7 +30,13 @@ import ( "github.com/pingcap/tidb/pkg/util/chunk" ) -func encodeRowChangedEvent(e *commonEvent.RowEvent, config *common.Config, largeMessageOnlyHandleKeyColumns bool, claimCheckLocationName string) ([]byte, []byte, int, error) { +func encodeRowChangedEvent( + e *commonEvent.RowEvent, + columnFlags map[string]uint64, + config *common.Config, + largeMessageOnlyHandleKeyColumns bool, + claimCheckLocationName string, +) ([]byte, []byte, int, error) { var ( keyBuf bytes.Buffer valueBuf bytes.Buffer @@ -57,30 +63,30 @@ func encodeRowChangedEvent(e *commonEvent.RowEvent, config *common.Config, large onlyHandleKeyColumns := config.DeleteOnlyHandleKeyColumns || largeMessageOnlyHandleKeyColumns valueWriter.WriteObject(func() { valueWriter.WriteObjectField("d", func() { - err = writeColumnFieldValues(valueWriter, e.GetPreRows(), e.TableInfo, e.ColumnSelector, onlyHandleKeyColumns) + err = writeColumnFieldValues(valueWriter, e.GetPreRows(), e.TableInfo, columnFlags, e.ColumnSelector, onlyHandleKeyColumns) }) }) } else if e.IsInsert() { valueWriter.WriteObject(func() { valueWriter.WriteObjectField("u", func() { - err = writeColumnFieldValues(valueWriter, e.GetRows(), e.TableInfo, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) + err = writeColumnFieldValues(valueWriter, e.GetRows(), e.TableInfo, columnFlags, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) }) }) } else if e.IsUpdate() { valueWriter.WriteObject(func() { valueWriter.WriteObjectField("u", func() { - err = writeColumnFieldValues(valueWriter, e.GetRows(), e.TableInfo, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) + err = writeColumnFieldValues(valueWriter, e.GetRows(), e.TableInfo, columnFlags, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) }) if err != nil { return } if !config.OnlyOutputUpdatedColumns { valueWriter.WriteObjectField("p", func() { - err = writeColumnFieldValues(valueWriter, e.GetPreRows(), e.TableInfo, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) + err = writeColumnFieldValues(valueWriter, e.GetPreRows(), e.TableInfo, columnFlags, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) }) } else { valueWriter.WriteObjectField("p", func() { - writeUpdatedColumnFieldValues(valueWriter, e.GetPreRows(), e.GetRows(), e.TableInfo, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) + writeUpdatedColumnFieldValues(valueWriter, e.GetPreRows(), e.GetRows(), e.TableInfo, columnFlags, e.ColumnSelector, largeMessageOnlyHandleKeyColumns) }) } }) @@ -163,26 +169,24 @@ func writeColumnFieldValue( col *model.ColumnInfo, row *chunk.Row, idx int, - tableInfo *commonType.TableInfo, + isHandle bool, + columnFlag uint64, ) { - colType := col.GetType() - flag := col.GetFlag() - whereHandle := tableInfo.IsHandleKey(col.ID) - - writer.WriteIntField("t", int(colType)) - if whereHandle { - writer.WriteBoolField("h", whereHandle) + fieldType := col.FieldType + writer.WriteIntField("t", int(fieldType.GetType())) + if isHandle { + writer.WriteBoolField("h", isHandle) } - writer.WriteUint64Field("f", uint64(flag)) + writer.WriteUint64Field("f", columnFlag) if row.IsNull(idx) { writer.WriteNullField("v") return } - switch col.GetType() { + switch fieldType.GetType() { case mysql.TypeBit: - d := row.GetDatum(idx, &col.FieldType) + d := row.GetDatum(idx, &fieldType) if d.IsNull() { writer.WriteNullField("v") } else { @@ -203,7 +207,7 @@ func writeColumnFieldValue( if len(value) == 0 { writer.WriteNullField("v") } else { - if mysql.HasBinaryFlag(flag) { + if mysql.HasBinaryFlag(fieldType.GetFlag()) { str := string(value) str = strconv.Quote(str) str = str[1 : len(str)-1] @@ -248,7 +252,7 @@ func writeColumnFieldValue( writer.WriteStringField("v", value.String()) } default: - d := row.GetDatum(idx, &col.FieldType) + d := row.GetDatum(idx, &fieldType) // NOTICE: GetValue() may return some types that go sql not support, which will cause sink DML fail // Make specified convert upper if you need // Go sql support type ref to: https://github.com/golang/go/blob/go1.17.4/src/database/sql/driver/types.go#L236 @@ -261,25 +265,25 @@ func writeColumnFieldValues( jWriter *util.JSONWriter, row *chunk.Row, tableInfo *commonType.TableInfo, + columnFlags map[string]uint64, selector columnselector.Selector, onlyHandleKeyColumns bool, ) error { - flag := false // flag to check if any column is written - + var encoded bool colInfo := tableInfo.GetColumns() - for idx, col := range colInfo { if selector.Select(col) { - if onlyHandleKeyColumns && !tableInfo.IsHandleKey(col.ID) { + isHandle := tableInfo.IsHandleKey(col.ID) + if onlyHandleKeyColumns && !isHandle { continue } - flag = true + encoded = true jWriter.WriteObjectField(col.Name.O, func() { - writeColumnFieldValue(jWriter, col, row, idx, tableInfo) + writeColumnFieldValue(jWriter, col, row, idx, isHandle, columnFlags[col.Name.O]) }) } } - if !flag { + if !encoded { return errors.ErrOpenProtocolCodecInvalidData.GenWithStack("not found handle key columns for the delete event") } return nil @@ -290,6 +294,7 @@ func writeUpdatedColumnFieldValues( preRow *chunk.Row, row *chunk.Row, tableInfo *commonType.TableInfo, + columnFlags map[string]uint64, selector columnselector.Selector, onlyHandleKeyColumns bool, ) { @@ -299,10 +304,11 @@ func writeUpdatedColumnFieldValues( for idx, col := range colInfo { if selector.Select(col) { - if onlyHandleKeyColumns && !tableInfo.IsHandleKey(col.ID) { + isHandle := tableInfo.IsHandleKey(col.ID) + if onlyHandleKeyColumns && !isHandle { continue } - writeColumnFieldValueIfUpdated(jWriter, col, preRow, row, idx, tableInfo) + writeColumnFieldValueIfUpdated(jWriter, col, preRow, row, idx, isHandle, columnFlags[col.Name.O]) } } } @@ -313,17 +319,17 @@ func writeColumnFieldValueIfUpdated( preRow *chunk.Row, row *chunk.Row, idx int, - tableInfo *commonType.TableInfo, + isHandle bool, + columnFlag uint64, ) { colType := col.GetType() flag := col.GetFlag() - whereHandle := tableInfo.IsHandleKey(col.ID) writeFunc := func(writeColumnValue func()) { writer.WriteObjectField(col.Name.O, func() { writer.WriteIntField("t", int(colType)) - if whereHandle { - writer.WriteBoolField("h", whereHandle) + if isHandle { + writer.WriteBoolField("h", isHandle) } writer.WriteUint64Field("f", uint64(flag)) writeColumnValue() @@ -338,7 +344,7 @@ func writeColumnFieldValueIfUpdated( return } if !preRow.IsNull(idx) && row.IsNull(idx) { - writeColumnFieldValue(writer, col, preRow, idx, tableInfo) + writeColumnFieldValue(writer, col, preRow, idx, isHandle, columnFlag) return } diff --git a/pkg/sink/codec/open/decoder.go b/pkg/sink/codec/open/decoder.go index 837ccd53d3..667a906cfe 100644 --- a/pkg/sink/codec/open/decoder.go +++ b/pkg/sink/codec/open/decoder.go @@ -251,7 +251,7 @@ func buildColumns( Value: value, } if _, ok := handleKeyColumns[name]; ok { - col.Flag = mysql.PriKeyFlag + col.Flag |= binaryFlag } result[name] = col } @@ -379,26 +379,24 @@ func newTiColumns(rawColumns map[string]column) []*timodel.ColumnInfo { col.Name = pmodel.NewCIStr(name) col.FieldType = *types.NewFieldType(raw.Type) - if mysql.HasPriKeyFlag(raw.Flag) { + if isPrimary(raw.Flag) { col.AddFlag(mysql.PriKeyFlag) col.AddFlag(mysql.UniqueKeyFlag) col.AddFlag(mysql.NotNullFlag) } - if mysql.HasUnsignedFlag(raw.Flag) { + if isUnsigned(raw.Flag) { col.AddFlag(mysql.UnsignedFlag) } - if mysql.HasBinaryFlag(raw.Flag) { + if isBinary(raw.Flag) { col.AddFlag(mysql.BinaryFlag) + col.SetCharset("binary") + col.SetCollate("binary") } switch col.GetType() { case mysql.TypeVarchar, mysql.TypeString, mysql.TypeTinyBlob, mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob: - if mysql.HasBinaryFlag(col.GetFlag()) { - col.AddFlag(mysql.BinaryFlag) - col.SetCharset("binary") - col.SetCollate("binary") - } else { + if !mysql.HasBinaryFlag(col.GetFlag()) { col.SetCharset("utf8mb4") col.SetCollate("utf8mb4_bin") } diff --git a/pkg/sink/codec/open/encoder.go b/pkg/sink/codec/open/encoder.go index 3414486ff1..17a0bee64e 100644 --- a/pkg/sink/codec/open/encoder.go +++ b/pkg/sink/codec/open/encoder.go @@ -40,6 +40,8 @@ type batchEncoder struct { claimCheck *claimcheck.ClaimCheck + columnFlagsCache map[int64]map[string]uint64 + config *common.Config } @@ -50,8 +52,9 @@ func NewBatchEncoder(ctx context.Context, config *common.Config) (common.EventEn return nil, errors.Trace(err) } return &batchEncoder{ - config: config, - claimCheck: claimCheck, + config: config, + claimCheck: claimCheck, + columnFlagsCache: make(map[int64]map[string]uint64, 32), }, nil } @@ -61,13 +64,23 @@ func (d *batchEncoder) Clean() { } } +func (d *batchEncoder) fetchColumnFlags(e *commonEvent.RowEvent) map[string]uint64 { + result, ok := d.columnFlagsCache[e.GetTableID()] + if !ok { + result = initColumnFlags(e.TableInfo) + d.columnFlagsCache[e.GetTableID()] = result + } + return result +} + // AppendRowChangedEvent implements the RowEventEncoder interface func (d *batchEncoder) AppendRowChangedEvent( ctx context.Context, _ string, e *commonEvent.RowEvent, ) error { - key, value, length, err := encodeRowChangedEvent(e, d.config, false, "") + columnFlags := d.fetchColumnFlags(e) + key, value, length, err := encodeRowChangedEvent(e, columnFlags, d.config, false, "") if err != nil { return errors.Trace(err) } @@ -93,7 +106,7 @@ func (d *batchEncoder) AppendRowChangedEvent( return errors.Trace(err) } - key, value, length, err = encodeRowChangedEvent(e, d.config, true, d.claimCheck.FileNameWithPrefix(claimCheckFileName)) + key, value, length, err = encodeRowChangedEvent(e, columnFlags, d.config, true, d.claimCheck.FileNameWithPrefix(claimCheckFileName)) if err != nil { return errors.Trace(err) } @@ -110,7 +123,7 @@ func (d *batchEncoder) AppendRowChangedEvent( if d.config.LargeMessageHandle.HandleKeyOnly() { // it must that `LargeMessageHandle == LargeMessageHandleOnlyHandleKeyColumns` here. - key, value, length, err = encodeRowChangedEvent(e, d.config, true, "") + key, value, length, err = encodeRowChangedEvent(e, columnFlags, d.config, true, "") if err != nil { return errors.Trace(err) } @@ -211,6 +224,7 @@ func enhancedKeyValue(key, value []byte) ([]byte, []byte) { } func (d *batchEncoder) EncodeDDLEvent(e *commonEvent.DDLEvent) (*common.Message, error) { + delete(d.columnFlagsCache, e.TableID) key, value, err := encodeDDLEvent(e, d.config) if err != nil { return nil, errors.Trace(err) diff --git a/pkg/sink/codec/open/encoder_test.go b/pkg/sink/codec/open/encoder_test.go index 9ce61743ed..7f1de5a5b3 100644 --- a/pkg/sink/codec/open/encoder_test.go +++ b/pkg/sink/codec/open/encoder_test.go @@ -26,6 +26,71 @@ import ( "github.com/stretchr/testify/require" ) +func TestEncodeFlag(t *testing.T) { + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + helper.Tk().MustExec("use test") + + createTableDDL := `create table t( + a int primary key, + b int not null, + c int, + d int unsigned, + e blob, + unique key idx(b, c), + key idx2(c, d) + )` + job := helper.DDL2Job(createTableDDL) + tableInfo := helper.GetTableInfo(job) + + dmlEvent := helper.DML2Event("test", "t", + `insert into t values (1, 2, 3, 4, "0x010201")`) + row, ok := dmlEvent.GetNextRow() + require.True(t, ok) + + columnSelector := columnselector.NewDefaultColumnSelector() + + insertEvent := &commonEvent.RowEvent{ + TableInfo: tableInfo, + CommitTs: dmlEvent.GetCommitTs(), + Event: row, + ColumnSelector: columnSelector, + Callback: func() {}, + } + + ctx := context.Background() + codecConfig := common.NewConfig(config.ProtocolOpen) + + enc, err := NewBatchEncoder(ctx, codecConfig) + require.NoError(t, err) + + err = enc.AppendRowChangedEvent(ctx, "", insertEvent) + require.NoError(t, err) + + messages := enc.Build() + require.Len(t, messages, 1) + require.NotEmpty(t, messages[0]) + + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) + + err = decoder.AddKeyValue(messages[0].Key, messages[0].Value) + require.NoError(t, err) + + messageType, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, common.MessageTypeRow, messageType) + + decoded, err := decoder.NextDMLEvent() + require.NoError(t, err) + + change, ok := decoded.GetNextRow() + require.True(t, ok) + common.CompareRow(t, insertEvent.Event, insertEvent.TableInfo, change, decoded.TableInfo) +} + func TestIntegerTypes(t *testing.T) { helper := commonEvent.NewEventTestHelper(t) defer helper.Close() @@ -84,41 +149,36 @@ func TestIntegerTypes(t *testing.T) { ctx := context.Background() codecConfig := common.NewConfig(config.ProtocolOpen) - for _, enableTiDBExtension := range []bool{true, false} { - for _, event := range []*commonEvent.RowEvent{minValueEvent, maxValueEvent} { - codecConfig.EnableTiDBExtension = enableTiDBExtension - encoder, err := NewBatchEncoder(ctx, codecConfig) - require.NoError(t, err) + for _, event := range []*commonEvent.RowEvent{minValueEvent, maxValueEvent} { + encoder, err := NewBatchEncoder(ctx, codecConfig) + require.NoError(t, err) - err = encoder.AppendRowChangedEvent(ctx, "", event) - require.NoError(t, err) + err = encoder.AppendRowChangedEvent(ctx, "", event) + require.NoError(t, err) - messages := encoder.Build() - require.Len(t, messages, 1) + messages := encoder.Build() + require.Len(t, messages, 1) - decoder, err := NewBatchDecoder(ctx, codecConfig, nil) - require.NoError(t, err) + decoder, err := NewBatchDecoder(ctx, codecConfig, nil) + require.NoError(t, err) - err = decoder.AddKeyValue(messages[0].Key, messages[0].Value) - require.NoError(t, err) + err = decoder.AddKeyValue(messages[0].Key, messages[0].Value) + require.NoError(t, err) - messageType, hasNext, err := decoder.HasNext() - require.NoError(t, err) - require.True(t, hasNext) - require.Equal(t, common.MessageTypeRow, messageType) + messageType, hasNext, err := decoder.HasNext() + require.NoError(t, err) + require.True(t, hasNext) + require.Equal(t, common.MessageTypeRow, messageType) - decoded, err := decoder.NextDMLEvent() - require.NoError(t, err) + decoded, err := decoder.NextDMLEvent() + require.NoError(t, err) - if enableTiDBExtension { - require.Equal(t, event.CommitTs, decoded.GetCommitTs()) - } + require.Equal(t, event.CommitTs, decoded.GetCommitTs()) - change, ok := decoded.GetNextRow() - require.True(t, ok) + change, ok := decoded.GetNextRow() + require.True(t, ok) - common.CompareRow(t, event.Event, event.TableInfo, change, decoded.TableInfo) - } + common.CompareRow(t, event.Event, event.TableInfo, change, decoded.TableInfo) } } @@ -706,6 +766,7 @@ func TestMessageTooLarge(t *testing.T) { err = encoder.AppendRowChangedEvent(ctx, "", insertRowEvent) require.ErrorIs(t, err, errors.ErrMessageTooLarge) + require.Equal(t, count, 0) } func TestLargeMessageWithHandleEnableHandleKeyOnly(t *testing.T) { diff --git a/pkg/sink/codec/open/message.go b/pkg/sink/codec/open/message.go index d033d78807..435936c9e9 100644 --- a/pkg/sink/codec/open/message.go +++ b/pkg/sink/codec/open/message.go @@ -20,6 +20,7 @@ import ( "strconv" "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/parser/types" @@ -53,9 +54,9 @@ func (m *messageKey) Decode(data []byte) { type column struct { Type byte `json:"t"` // Deprecated: please use Flag instead. - WhereHandle *bool `json:"h,omitempty"` - Flag uint `json:"f"` - Value any `json:"v"` + WhereHandle *bool `json:"h,omitempty"` + Flag uint64 `json:"f"` + Value any `json:"v"` } // formatColumn formats a codec column. @@ -64,7 +65,7 @@ func formatColumn(c column, ft types.FieldType) column { switch c.Type { case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: str := c.Value.(string) - if mysql.HasBinaryFlag(c.Flag) { + if isBinary(c.Flag) { str, err = strconv.Unquote("\"" + str + "\"") if err != nil { log.Panic("invalid column value, please report a bug", zap.Any("value", str), zap.Error(err)) @@ -93,7 +94,7 @@ func formatColumn(c column, ft types.FieldType) column { } case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24: if s, ok := c.Value.(json.Number); ok { - if mysql.HasUnsignedFlag(c.Flag) { + if isUnsigned(c.Flag) { c.Value, err = strconv.ParseUint(s.String(), 10, 64) } else { c.Value, err = strconv.ParseInt(s.String(), 10, 64) @@ -103,7 +104,7 @@ func formatColumn(c column, ft types.FieldType) column { } // is it possible be the float64? } else if f, ok := c.Value.(float64); ok { - if mysql.HasUnsignedFlag(c.Flag) { + if isUnsigned(c.Flag) { c.Value = uint64(f) } else { c.Value = int64(f) @@ -206,3 +207,117 @@ func (m *messageRow) decode(data []byte) { log.Panic("decode message row failed", zap.Any("data", data), zap.Error(err)) } } + +const ( + // binaryFlag means the column charset is binary + binaryFlag uint64 = 1 << iota + + // handleKeyFlag means the column is selected as the handle key + // The handleKey is chosen by the following rules in the order: + // 1. if the table has primary key, it's the handle key. + // 2. If the table has not null unique key, it's the handle key. + // 3. If the table has no primary key and no not null unique key, it has no handleKey. + handleKeyFlag + + // generatedColumnFlag means the column is a generated column + generatedColumnFlag + + // primaryKeyFlag means the column is primary key + primaryKeyFlag + + // uniqueKeyFlag means the column is unique key + uniqueKeyFlag + + // multipleKeyFlag means the column is multiple key + multipleKeyFlag + + // nullableFlag means the column is nullable + nullableFlag + + // unsignedFlag means the column stores an unsigned integer + unsignedFlag +) + +func isBinary(flag uint64) bool { + return flag&binaryFlag != 0 +} + +func isPrimary(flag uint64) bool { + return flag&primaryKeyFlag != 0 +} + +func isUnique(flag uint64) bool { + return flag&uniqueKeyFlag != 0 +} + +func isMultiKey(flag uint64) bool { + return flag&multipleKeyFlag != 0 +} + +func isNullable(flag uint64) bool { + return flag&nullableFlag != 0 +} + +func isUnsigned(flag uint64) bool { + return flag&unsignedFlag != 0 +} + +func initColumnFlags(tableInfo *commonType.TableInfo) map[string]uint64 { + result := make(map[string]uint64, len(tableInfo.GetColumns())) + for _, col := range tableInfo.GetColumns() { + var flag uint64 + if col.GetCharset() == "binary" { + flag |= binaryFlag + } + origin := col.GetFlag() + if col.IsGenerated() { + flag |= generatedColumnFlag + } + if mysql.HasUniKeyFlag(origin) { + flag |= uniqueKeyFlag + } + if mysql.HasPriKeyFlag(origin) { + flag |= primaryKeyFlag + if tableInfo.PKIsHandle() { + flag |= handleKeyFlag + } + } + if !mysql.HasNotNullFlag(origin) { + flag |= nullableFlag + } + if mysql.HasMultipleKeyFlag(origin) { + flag |= multipleKeyFlag + } + if mysql.HasUnsignedFlag(origin) { + flag |= unsignedFlag + } + result[col.Name.O] = flag + } + + // In TiDB, just as in MySQL, only the first column of an index can be marked as "multiple key" or "unique key", + // and only the first column of a unique index may be marked as "unique key". + // See https://dev.mysql.com/doc/refman/5.7/en/show-columns.html. + // Yet if an index has multiple columns, we would like to easily determine that all those columns are indexed, + // which is crucial for the completeness of the information we pass to the downstream. + // Therefore, instead of using the MySQL standard, + // we made our own decision to mark all columns in an index with the appropriate flag(s). + for _, idxInfo := range tableInfo.GetIndices() { + for _, idxCol := range idxInfo.Columns { + flag := result[idxCol.Name.O] + if idxInfo.Primary { + flag |= primaryKeyFlag + } else if idxInfo.Unique { + flag |= uniqueKeyFlag + } + if len(idxInfo.Columns) > 1 { + flag |= multipleKeyFlag + } + colID := tableInfo.ForceGetColumnIDByName(idxCol.Name.O) + if tableInfo.IsHandleKey(colID) { + flag |= handleKeyFlag + } + result[idxCol.Name.O] = flag + } + } + return result +} From f188360afd4d1456f1600798892fa04afdfde00a Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Thu, 17 Apr 2025 11:04:06 +0800 Subject: [PATCH 07/13] dependencies: bump sarama to fix the kafka cluste stuck and data race panic problem (#1237) close pingcap/ticdc#1236 --- go.mod | 16 +++++++++------- go.sum | 31 ++++++++++++++++--------------- 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/go.mod b/go.mod index e5fb60cef4..c4d24924ce 100644 --- a/go.mod +++ b/go.mod @@ -76,12 +76,12 @@ require ( go.uber.org/mock v0.4.0 go.uber.org/multierr v1.11.0 go.uber.org/zap v1.27.0 - golang.org/x/net v0.33.0 + golang.org/x/net v0.39.0 golang.org/x/oauth2 v0.24.0 - golang.org/x/sync v0.10.0 - golang.org/x/sys v0.28.0 - golang.org/x/term v0.27.0 - golang.org/x/text v0.21.0 + golang.org/x/sync v0.13.0 + golang.org/x/sys v0.32.0 + golang.org/x/term v0.31.0 + golang.org/x/text v0.24.0 golang.org/x/time v0.7.0 google.golang.org/grpc v1.65.0 google.golang.org/protobuf v1.35.2 @@ -200,7 +200,7 @@ require ( github.com/grpc-ecosystem/grpc-gateway v1.16.0 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 // indirect github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c // indirect - github.com/hashicorp/errwrap v1.0.0 // indirect + github.com/hashicorp/errwrap v1.1.0 // indirect github.com/hashicorp/go-multierror v1.1.1 // indirect github.com/hashicorp/go-uuid v1.0.3 // indirect github.com/improbable-eng/grpc-web v0.12.0 // indirect @@ -336,7 +336,7 @@ require ( go.opentelemetry.io/otel/trace v1.24.0 // indirect go.opentelemetry.io/proto/otlp v1.1.0 // indirect golang.org/x/arch v0.3.0 // indirect - golang.org/x/crypto v0.31.0 // indirect + golang.org/x/crypto v0.37.0 // indirect golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 // indirect golang.org/x/mod v0.22.0 // indirect golang.org/x/tools v0.28.0 // indirect @@ -363,3 +363,5 @@ require ( // copy from TiDB replace github.com/go-ldap/ldap/v3 v3.4.4 => github.com/yangkeao/ldap/v3 v3.4.5-0.20230421065457-369a3bab1117 + +replace github.com/IBM/sarama v1.41.2 => github.com/pingcap/sarama v1.41.2-pingcap-20250416 diff --git a/go.sum b/go.sum index 427656048b..a33f24f18a 100644 --- a/go.sum +++ b/go.sum @@ -66,8 +66,6 @@ github.com/DataDog/zstd v1.5.5 h1:oWf5W7GtOLgp6bciQYDmhHHjdhYkALu6S/5Ni9ZgSvQ= github.com/DataDog/zstd v1.5.5/go.mod h1:g4AWEaM3yOg3HYfnJ3YIawPnVdXJh9QME85blwSAmyw= github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= github.com/HdrHistogram/hdrhistogram-go v1.1.2/go.mod h1:yDgFjdqOqDEKOvasDdhWNXYg9BVp4O+o5f6V/ehm6Oo= -github.com/IBM/sarama v1.41.2 h1:ZDBZfGPHAD4uuAtSv4U22fRZBgst0eEwGFzLj0fb85c= -github.com/IBM/sarama v1.41.2/go.mod h1:xdpu7sd6OE1uxNdjYTSKUfY8FaKkJES9/+EyjSgiGQk= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/KimMachineGun/automemlimit v0.2.4 h1:GBty8TK8k0aJer1Pq5/3Vdt2ef+YpLhcqNo+PSD5CoI= @@ -536,8 +534,9 @@ github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1 h1:/c3QmbOGMGTOumP2iT/rCwB7b0Q github.com/grpc-ecosystem/grpc-gateway/v2 v2.19.1/go.mod h1:5SN9VR2LTsRFsrEC6FHgRbTWrTHu6tqPeKxEQv15giM= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c h1:6rhixN/i8ZofjG1Y75iExal34USq5p+wiN1tpie8IrU= github.com/gsterjov/go-libsecret v0.0.0-20161001094733-a6f4afe4910c/go.mod h1:NMPJylDgVpX0MLRlPy15sqSwOFv/U1GZ2m21JhFfek0= -github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= +github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= +github.com/hashicorp/errwrap v1.1.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= github.com/hashicorp/go-multierror v1.1.1/go.mod h1:iw975J/qwKPdAO1clOe2L8331t/9/fmwbPZ6JB6eMoM= github.com/hashicorp/go-uuid v0.0.0-20180228145832-27454136f036/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= @@ -821,6 +820,8 @@ github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuR github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= github.com/pingcap/log v1.1.1-0.20241212030209-7e3ff8601a2a h1:WIhmJBlNGmnCWH6TLMdZfNEDaiU8cFpZe3iaqDbQ0M8= github.com/pingcap/log v1.1.1-0.20241212030209-7e3ff8601a2a/go.mod h1:ORfBOFp1eteu2odzsyaxI+b8TzJwgjwyQcGhI+9SfEA= +github.com/pingcap/sarama v1.41.2-pingcap-20250416 h1:bmoJVev5VvlceFq5ls02XMgzy6uzilv41+h92UjYuGo= +github.com/pingcap/sarama v1.41.2-pingcap-20250416/go.mod h1:Kwi9CT6CuDYad3KS7HqjsbmD2DWkIKI7qI6a8PKlGb4= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 h1:T4pXRhBflzDeAhmOQHNPRRogMYxP13V7BkYw3ZsoSfE= github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5/go.mod h1:rlimy0GcTvjiJqvD5mXTRr8O2eNZPBrcUgiWVYp9530= github.com/pingcap/tidb v1.1.0-beta.0.20241223052309-3735ed55a394 h1:I7if6LVyQ/hOXiGWP88jZx67x4Lz2YwdaXME9DmaiDo= @@ -1196,8 +1197,8 @@ golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0 golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= -golang.org/x/crypto v0.31.0 h1:ihbySMvVjLAeSH1IbfcRTkD/iNscyz8rGzjF/E5hV6U= -golang.org/x/crypto v0.31.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/crypto v0.37.0 h1:kJNSjF/Xp7kU0iB2Z+9viTPMW4EqqsrywMXLJOOsXSE= +golang.org/x/crypto v0.37.0/go.mod h1:vg+k43peMZ0pUMhYmVAWysMK35e6ioLh3wB8ZCAfbVc= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1295,8 +1296,8 @@ golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= -golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= -golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= +golang.org/x/net v0.39.0 h1:ZCu7HMWDxpXpaiKdhzIfaltL9Lp31x/3fCP11bc6/fY= +golang.org/x/net v0.39.0/go.mod h1:X7NRbYVEA+ewNkCNyJ513WmMdQ3BineSwVtN2zD/d+E= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1314,8 +1315,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= -golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.13.0 h1:AauUjRAJ9OSnvULf/ARrrVywoJDy0YS2AwQ98I37610= +golang.org/x/sync v0.13.0/go.mod h1:1dzgHSNfp02xaA81J2MS99Qcpr2w7fw1gpm99rleRqA= golang.org/x/sys v0.0.0-20180816055513-1c9583448a9c/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1380,8 +1381,8 @@ golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= -golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.32.0 h1:s77OFDvIQeibCmezSnk/q6iAfkdiQaJi4VzroCFrN20= +golang.org/x/sys v0.32.0/go.mod h1:BJP2sWEmIv4KK5OTEluFJCKSidICx8ciO85XgH3Ak8k= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1389,8 +1390,8 @@ golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.6.0/go.mod h1:m6U89DPEgQRMq3DNkDClhWw02AUbt2daBVO4cn4Hv9U= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.13.0/go.mod h1:LTmsnFJwVN6bCy1rVCoS+qHT1HhALEFxKncY3WNNh4U= -golang.org/x/term v0.27.0 h1:WP60Sv1nlK1T6SupCHbXzSaN0b9wUmsPoRS9b61A23Q= -golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= +golang.org/x/term v0.31.0 h1:erwDkOK1Msy6offm1mOgvspSkslFnIGsFnxOKoufg3o= +golang.org/x/term v0.31.0/go.mod h1:R4BeIy7D95HzImkxGkTW1UQTtP54tio2RyHz7PwK0aw= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1405,8 +1406,8 @@ golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.9.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/text v0.13.0/go.mod h1:TvPlkZtksWOMsz7fbANvkp4WM8x/WCo/om8BMLbz+aE= -golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= -golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +golang.org/x/text v0.24.0 h1:dd5Bzh4yt5KYA8f9CJHCP4FB4D51c2c6JvN37xJJkJ0= +golang.org/x/text v0.24.0/go.mod h1:L8rBsPeo2pSS+xqN0d5u2ikmjtmoJbDBT1b7nHvFCdU= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= From b2559055a0441ea5d960c601a8d73d9074f686f8 Mon Sep 17 00:00:00 2001 From: nhsmw Date: Thu, 17 Apr 2025 16:31:48 +0800 Subject: [PATCH 08/13] sink: Treat column and index names as case insensitive (#1223) ref pingcap/tiflow#12103 --- .../sink/eventrouter/partition/column_test.go | 48 ++++++++++ .../sink/eventrouter/partition/columns.go | 3 +- .../sink/eventrouter/partition/index_value.go | 2 +- .../eventrouter/partition/index_value_test.go | 94 +++++++++++++++++++ pkg/common/table_info.go | 14 ++- pkg/common/table_info_helper_test.go | 82 ++++++++++++++++ .../mq_sink_dispatcher/conf/changefeed.toml | 3 +- .../mq_sink_dispatcher/conf/diff_config.toml | 2 +- .../conf/new_changefeed.toml | 3 +- .../mq_sink_dispatcher/run.sh | 9 ++ 10 files changed, 251 insertions(+), 9 deletions(-) create mode 100644 downstreamadapter/sink/eventrouter/partition/column_test.go create mode 100644 downstreamadapter/sink/eventrouter/partition/index_value_test.go diff --git a/downstreamadapter/sink/eventrouter/partition/column_test.go b/downstreamadapter/sink/eventrouter/partition/column_test.go new file mode 100644 index 0000000000..8aac33051b --- /dev/null +++ b/downstreamadapter/sink/eventrouter/partition/column_test.go @@ -0,0 +1,48 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "testing" + + "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/stretchr/testify/require" +) + +func TestColumnsDispatcher(t *testing.T) { + helper := event.NewEventTestHelper(t) + defer helper.Close() + helper.Tk().MustExec("use test") + job := helper.DDL2Job("create table t1(col1 int, col2 int, col3 int)") + require.NotNil(t, job) + tableInfo := helper.GetTableInfo(job) + dml := helper.DML2Event("test", "t1", "insert into t1 values(22, 11, 33)") + row, exist := dml.GetNextRow() + require.True(t, exist) + + p := newColumnsPartitionGenerator([]string{"col-2", "col-not-found"}) + _, _, err := p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 1) + require.ErrorIs(t, err, errors.ErrDispatcherFailed) + + p = newColumnsPartitionGenerator([]string{"col2", "col1"}) + index, _, err := p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 1) + require.NoError(t, err) + require.Equal(t, int32(15), index) + + p = newColumnsPartitionGenerator([]string{"COL2", "Col1"}) + index, _, err = p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 1) + require.NoError(t, err) + require.Equal(t, int32(15), index) +} diff --git a/downstreamadapter/sink/eventrouter/partition/columns.go b/downstreamadapter/sink/eventrouter/partition/columns.go index d12920e2f1..bc4797b839 100644 --- a/downstreamadapter/sink/eventrouter/partition/columns.go +++ b/downstreamadapter/sink/eventrouter/partition/columns.go @@ -15,6 +15,7 @@ package partition import ( "strconv" + "strings" "sync" "github.com/pingcap/log" @@ -70,7 +71,7 @@ func (r *ColumnsPartitionGenerator) GeneratePartitionIndexAndKey(row *commonEven if value == nil { continue } - r.hasher.Write([]byte(r.Columns[idx]), []byte(model.ColumnValueString(value))) + r.hasher.Write([]byte(strings.ToLower(r.Columns[idx])), []byte(model.ColumnValueString(value))) } sum32 := r.hasher.Sum32() diff --git a/downstreamadapter/sink/eventrouter/partition/index_value.go b/downstreamadapter/sink/eventrouter/partition/index_value.go index 6cc389c51c..cedab483d9 100644 --- a/downstreamadapter/sink/eventrouter/partition/index_value.go +++ b/downstreamadapter/sink/eventrouter/partition/index_value.go @@ -63,7 +63,7 @@ func (r *IndexValuePartitionGenerator) GeneratePartitionIndexAndKey(row *commonE } if tableInfo.IsHandleKey(col.ID) { value := common.ExtractColVal(&rowData, col, idx) - r.hasher.Write([]byte(col.Name.O), []byte(model.ColumnValueString(value))) + r.hasher.Write([]byte(col.Name.L), []byte(model.ColumnValueString(value))) } } } else { diff --git a/downstreamadapter/sink/eventrouter/partition/index_value_test.go b/downstreamadapter/sink/eventrouter/partition/index_value_test.go new file mode 100644 index 0000000000..f898bb3164 --- /dev/null +++ b/downstreamadapter/sink/eventrouter/partition/index_value_test.go @@ -0,0 +1,94 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package partition + +import ( + "testing" + + "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/stretchr/testify/require" +) + +func genRow(t *testing.T, helper *event.EventTestHelper, schema string, table string, dml ...string) *event.RowChange { + event := helper.DML2Event(schema, table, dml...) + row, exist := event.GetNextRow() + require.True(t, exist) + helper.DDL2Job("TRUNCATE TABLE " + table) + return &row +} + +func TestIndexValueDispatcher(t *testing.T) { + helper := event.NewEventTestHelper(t) + defer helper.Close() + helper.Tk().MustExec("use test") + job1 := helper.DDL2Job("create table t1(a int primary key, b int)") + require.NotNil(t, job1) + job2 := helper.DDL2Job("create table t2(a int, b int, primary key(a,b))") + require.NotNil(t, job2) + tableInfoWithSinglePK := helper.GetTableInfo(job1) + tableInfoWithCompositePK := helper.GetTableInfo(job2) + + testCases := []struct { + row *event.RowChange + tableInfo *common.TableInfo + expectPartition int32 + }{ + {row: genRow(t, helper, "test", "t1", "insert into t1 values(11, 12)"), tableInfo: tableInfoWithSinglePK, expectPartition: 2}, + {row: genRow(t, helper, "test", "t1", "insert into t1 values(22, 22)"), tableInfo: tableInfoWithSinglePK, expectPartition: 11}, + {row: genRow(t, helper, "test", "t1", "insert into t1 values(11, 33)"), tableInfo: tableInfoWithSinglePK, expectPartition: 2}, + {row: genRow(t, helper, "test", "t2", "insert into t2 values(11, 22)"), tableInfo: tableInfoWithCompositePK, expectPartition: 5}, + {row: genRow(t, helper, "test", "t2", "insert into t2 (b, a) values(22, 11)"), tableInfo: tableInfoWithCompositePK, expectPartition: 5}, + {row: genRow(t, helper, "test", "t2", "insert into t2 values(11, 0)"), tableInfo: tableInfoWithCompositePK, expectPartition: 14}, + {row: genRow(t, helper, "test", "t2", "insert into t2 values(11, 33)"), tableInfo: tableInfoWithCompositePK, expectPartition: 2}, + } + p := newIndexValuePartitionGenerator("") + for _, tc := range testCases { + index, _, err := p.GeneratePartitionIndexAndKey(tc.row, 16, tc.tableInfo, 1) + require.Equal(t, tc.expectPartition, index) + require.NoError(t, err) + } +} + +func TestIndexValueDispatcherWithIndexName(t *testing.T) { + helper := event.NewEventTestHelper(t) + defer helper.Close() + helper.Tk().MustExec("use test") + job := helper.DDL2Job("create table t1(a int, INDEX index1(a))") + require.NotNil(t, job) + tableInfo := helper.GetTableInfo(job) + dml := helper.DML2Event("test", "t1", "insert into t1 values(11)") + row, exist := dml.GetNextRow() + require.True(t, exist) + + p := newIndexValuePartitionGenerator("index2") + _, _, err := p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 33) + require.ErrorIs(t, err, errors.ErrDispatcherFailed) + + p = newIndexValuePartitionGenerator("index1") + index, _, err := p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 33) + require.NoError(t, err) + require.Equal(t, int32(2), index) + + p = newIndexValuePartitionGenerator("INDEX1") + index, _, err = p.GeneratePartitionIndexAndKey(&row, 16, tableInfo, 33) + require.NoError(t, err) + require.Equal(t, int32(2), index) + + p = newIndexValuePartitionGenerator("") + index, _, err = p.GeneratePartitionIndexAndKey(&row, 3, tableInfo, 33) + require.NoError(t, err) + require.Equal(t, int32(0), index) +} diff --git a/pkg/common/table_info.go b/pkg/common/table_info.go index df209bda8f..7eb3c5075b 100644 --- a/pkg/common/table_info.go +++ b/pkg/common/table_info.go @@ -320,7 +320,7 @@ func (ti *TableInfo) HasVirtualColumns() bool { // GetIndex return the corresponding index by the given name. func (ti *TableInfo) GetIndex(name string) *model.IndexInfo { for _, index := range ti.columnSchema.Indices { - if index != nil && index.Name.O == name { + if index != nil && index.Name.L == strings.ToLower(name) { return index } } @@ -328,6 +328,9 @@ func (ti *TableInfo) GetIndex(name string) *model.IndexInfo { } // IndexByName returns the index columns and offsets of the corresponding index by name +// Column is not case-sensitive on any platform, nor are column aliases. +// So we always match in lowercase. +// See also: https://dev.mysql.com/doc/refman/5.7/en/identifier-case-sensitivity.html func (ti *TableInfo) IndexByName(name string) ([]string, []int, bool) { index := ti.GetIndex(name) if index == nil { @@ -336,7 +339,7 @@ func (ti *TableInfo) IndexByName(name string) ([]string, []int, bool) { names := make([]string, 0, len(index.Columns)) offset := make([]int, 0, len(index.Columns)) for _, col := range index.Columns { - names = append(names, col.Name.O) + names = append(names, col.Name.L) offset = append(offset, col.Offset) } return names, offset, true @@ -344,18 +347,21 @@ func (ti *TableInfo) IndexByName(name string) ([]string, []int, bool) { // OffsetsByNames returns the column offsets of the corresponding columns by names // If any column does not exist, return false +// Column is not case-sensitive on any platform, nor are column aliases. +// So we always match in lowercase. +// See also: https://dev.mysql.com/doc/refman/5.7/en/identifier-case-sensitivity.html func (ti *TableInfo) OffsetsByNames(names []string) ([]int, bool) { // todo: optimize it columnOffsets := make(map[string]int, len(ti.columnSchema.Columns)) for _, col := range ti.columnSchema.Columns { if col != nil { - columnOffsets[col.Name.O] = col.Offset + columnOffsets[col.Name.L] = ti.MustGetColumnOffsetByID(col.ID) } } result := make([]int, 0, len(names)) for _, col := range names { - offset, ok := columnOffsets[col] + offset, ok := columnOffsets[strings.ToLower(col)] if !ok { return nil, false } diff --git a/pkg/common/table_info_helper_test.go b/pkg/common/table_info_helper_test.go index 3b4659448e..f2ca4ba724 100644 --- a/pkg/common/table_info_helper_test.go +++ b/pkg/common/table_info_helper_test.go @@ -147,3 +147,85 @@ func TestColumnIndex(t *testing.T) { require.Equal(t, tableInfo.GetIndexColumns(), [][]int64{{101}, {102, 103}}) require.Equal(t, tableInfo.GetPKIndex(), []int64{101}) } + +func TestIndexByName(t *testing.T) { + tableInfo := WrapTableInfo("test", &model.TableInfo{ + Indices: nil, + }) + names, offsets, ok := tableInfo.IndexByName("idx1") + require.False(t, ok) + require.Nil(t, names) + require.Nil(t, offsets) + + tableInfo = WrapTableInfo("test", &model.TableInfo{ + Indices: []*model.IndexInfo{ + { + Name: pmodel.NewCIStr("idx1"), + Columns: []*model.IndexColumn{ + { + Name: pmodel.NewCIStr("col1"), + }, + }, + }, + }, + }) + + names, offsets, ok = tableInfo.IndexByName("idx2") + require.False(t, ok) + require.Nil(t, names) + require.Nil(t, offsets) + + names, offsets, ok = tableInfo.IndexByName("idx1") + require.True(t, ok) + require.Equal(t, []string{"col1"}, names) + require.Equal(t, []int{0}, offsets) + + names, offsets, ok = tableInfo.IndexByName("IDX1") + require.True(t, ok) + require.Equal(t, []string{"col1"}, names) + require.Equal(t, []int{0}, offsets) + + names, offsets, ok = tableInfo.IndexByName("Idx1") + require.True(t, ok) + require.Equal(t, []string{"col1"}, names) + require.Equal(t, []int{0}, offsets) +} + +func TestColumnsByNames(t *testing.T) { + tableInfo := WrapTableInfo("test", &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + Name: pmodel.NewCIStr("col2"), + ID: 1, + }, + { + Name: pmodel.NewCIStr("col1"), + ID: 0, + }, + { + Name: pmodel.NewCIStr("col3"), + ID: 2, + }, + }, + }) + + names := []string{"col1", "col2", "col3"} + offsets, ok := tableInfo.OffsetsByNames(names) + require.True(t, ok) + require.Equal(t, []int{1, 0, 2}, offsets) + + names = []string{"col2"} + offsets, ok = tableInfo.OffsetsByNames(names) + require.True(t, ok) + require.Equal(t, []int{0}, offsets) + + names = []string{"col1", "col-not-found"} + offsets, ok = tableInfo.OffsetsByNames(names) + require.False(t, ok) + require.Nil(t, offsets) + + names = []string{"Col1", "COL2", "CoL3"} + offsets, ok = tableInfo.OffsetsByNames(names) + require.True(t, ok) + require.Equal(t, []int{1, 0, 2}, offsets) +} diff --git a/tests/integration_tests/mq_sink_dispatcher/conf/changefeed.toml b/tests/integration_tests/mq_sink_dispatcher/conf/changefeed.toml index 8ee525511f..a4f6d601e6 100644 --- a/tests/integration_tests/mq_sink_dispatcher/conf/changefeed.toml +++ b/tests/integration_tests/mq_sink_dispatcher/conf/changefeed.toml @@ -1,5 +1,6 @@ [sink] dispatchers = [ {matcher = ['verify.t'], partition = "index-value"}, - {matcher = ['dispatcher.index'], partition = "index-value", index = "idx_a"} + {matcher = ['dispatcher.index'], partition = "index-value", index = "idx_a"}, + {matcher = ['dispatcher.columns'], partition = "columns", columns = ['id', 'col1']} ] diff --git a/tests/integration_tests/mq_sink_dispatcher/conf/diff_config.toml b/tests/integration_tests/mq_sink_dispatcher/conf/diff_config.toml index d3cb63bac5..8f8292a3f7 100644 --- a/tests/integration_tests/mq_sink_dispatcher/conf/diff_config.toml +++ b/tests/integration_tests/mq_sink_dispatcher/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] -output-dir = "/tmp/tidb_cdc_test/dispatcher/output" +output-dir = "/tmp/tidb_cdc_test/mq_sink_dispatcher/output" source-instances = ["tidb0"] diff --git a/tests/integration_tests/mq_sink_dispatcher/conf/new_changefeed.toml b/tests/integration_tests/mq_sink_dispatcher/conf/new_changefeed.toml index 4fbc029095..9fd5e983b2 100644 --- a/tests/integration_tests/mq_sink_dispatcher/conf/new_changefeed.toml +++ b/tests/integration_tests/mq_sink_dispatcher/conf/new_changefeed.toml @@ -1,4 +1,5 @@ [sink] dispatchers = [ - {matcher = ['dispatcher.index'], partition = "index-value", index = ""} + {matcher = ['dispatcher.index'], partition = "index-value", index = ""}, + {matcher = ['dispatcher.columns'], partition = "columns", columns = ['ID', 'CoL1']} ] diff --git a/tests/integration_tests/mq_sink_dispatcher/run.sh b/tests/integration_tests/mq_sink_dispatcher/run.sh index 6c234a6e5d..2f63d78984 100644 --- a/tests/integration_tests/mq_sink_dispatcher/run.sh +++ b/tests/integration_tests/mq_sink_dispatcher/run.sh @@ -42,8 +42,10 @@ function run() { run_sql "DROP DATABASE if exists dispatcher;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE DATABASE dispatcher;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "CREATE TABLE dispatcher.index (a int primary key, b int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE dispatcher.columns (id int primary key, col1 int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "INSERT INTO dispatcher.index values (1, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO dispatcher.columns values (1, 2);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} ensure $MAX_RETRIES check_changefeed_state http://${UP_PD_HOST_1}:${UP_PD_PORT_1} $changefeed_id "failed" "ErrDispatcherFailed" @@ -62,6 +64,13 @@ function run() { run_sql "UPDATE dispatcher.index set b = 6 where a = 2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "DELETE FROM dispatcher.index where a = 3;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + # make sure send rows into same partition + run_sql "INSERT INTO dispatcher.columns values (2, 3);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "INSERT INTO dispatcher.columns values (4, 5);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE dispatcher.columns set col1 = 3 where id = 1;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "UPDATE dispatcher.columns set CoL1 = 4 where ID = 2;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "DELETE FROM dispatcher.columns where Id = 4;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} + run_sql "CREATE TABLE test.finish_mark (a int primary key);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} # sync_diff can't check non-exist table, so we check expected tables are created in downstream first From 00d5c35101788986c64c2d44543d226b8cf31b97 Mon Sep 17 00:00:00 2001 From: nhsmw Date: Thu, 17 Apr 2025 17:14:43 +0800 Subject: [PATCH 09/13] codec: add csv protocol decoder (#1233) close pingcap/ticdc#1231 --- pkg/sink/codec/csv/csv_decoder.go | 226 +++++++++++++++++++++++++ pkg/sink/codec/csv/csv_decoder_test.go | 60 +++++++ pkg/sink/codec/csv/csv_message.go | 17 +- 3 files changed, 294 insertions(+), 9 deletions(-) create mode 100644 pkg/sink/codec/csv/csv_decoder.go create mode 100644 pkg/sink/codec/csv/csv_decoder_test.go diff --git a/pkg/sink/codec/csv/csv_decoder.go b/pkg/sink/codec/csv/csv_decoder.go new file mode 100644 index 0000000000..3cbaf84589 --- /dev/null +++ b/pkg/sink/codec/csv/csv_decoder.go @@ -0,0 +1,226 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package csv + +import ( + "context" + "encoding/base64" + "encoding/hex" + "fmt" + "io" + "strconv" + + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + lconfig "github.com/pingcap/tidb/pkg/lightning/config" + "github.com/pingcap/tidb/pkg/lightning/mydump" + "github.com/pingcap/tidb/pkg/lightning/worker" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/parser/charset" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/errors" +) + +const defaultIOConcurrency = 1 + +type batchDecoder struct { + codecConfig *common.Config + parser *mydump.CSVParser + data []byte + msg *csvMessage + tableInfo *commonType.TableInfo + closed bool +} + +// NewBatchDecoder creates a new BatchDecoder +func NewBatchDecoder(ctx context.Context, + codecConfig *common.Config, + tableInfo *commonType.TableInfo, + value []byte, +) (common.RowEventDecoder, error) { + var backslashEscape bool + + // if quote is not set in config, we should unespace backslash + // when parsing csv columns. + if len(codecConfig.Quote) == 0 { + backslashEscape = true + } + cfg := &lconfig.CSVConfig{ + Separator: codecConfig.Delimiter, + Delimiter: codecConfig.Quote, + Terminator: codecConfig.Terminator, + Null: []string{codecConfig.NullString}, + BackslashEscape: backslashEscape, + } + csvParser, err := mydump.NewCSVParser(ctx, cfg, + mydump.NewStringReader(string(value)), + int64(lconfig.ReadBlockSize), + worker.NewPool(ctx, defaultIOConcurrency, "io"), false, nil) + if err != nil { + return nil, err + } + return &batchDecoder{ + codecConfig: codecConfig, + tableInfo: tableInfo, + data: value, + msg: newCSVMessage(codecConfig), + parser: csvParser, + }, nil +} + +// AddKeyValue implements the RowEventDecoder interface. +func (b *batchDecoder) AddKeyValue(_, _ []byte) error { + return nil +} + +// HasNext implements the RowEventDecoder interface. +func (b *batchDecoder) HasNext() (common.MessageType, bool, error) { + err := b.parser.ReadRow() + if err != nil { + b.closed = true + if errors.Cause(err) == io.EOF { + return common.MessageTypeUnknown, false, nil + } + return common.MessageTypeUnknown, false, err + } + + row := b.parser.LastRow() + if err = b.msg.decode(row.Row); err != nil { + return common.MessageTypeUnknown, false, errors.Trace(err) + } + + return common.MessageTypeRow, true, nil +} + +// NextResolvedEvent implements the RowEventDecoder interface. +func (b *batchDecoder) NextResolvedEvent() (uint64, error) { + return 0, nil +} + +// NextDMLEvent implements the RowEventDecoder interface. +func (b *batchDecoder) NextDMLEvent() (*commonEvent.DMLEvent, error) { + if b.closed { + return nil, errors.WrapError(errors.ErrCSVDecodeFailed, errors.New("no csv row can be found")) + } + + e, err := csvMsg2RowChangedEvent(b.codecConfig, b.msg, b.tableInfo) + if err != nil { + return nil, errors.Trace(err) + } + return e, nil +} + +// NextDDLEvent implements the RowEventDecoder interface. +func (b *batchDecoder) NextDDLEvent() (*commonEvent.DDLEvent, error) { + return nil, nil +} + +func fromCsvValToColValue(csvConfig *common.Config, csvVal any, ft types.FieldType) (any, error) { + str, ok := csvVal.(string) + if !ok { + return csvVal, nil + } + var ( + val any + err error + ) + switch ft.GetType() { + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, + mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if ft.GetCharset() == charset.CharsetBin { + switch csvConfig.BinaryEncodingMethod { + case config.BinaryEncodingBase64: + return base64.StdEncoding.DecodeString(str) + case config.BinaryEncodingHex: + return hex.DecodeString(str) + default: + return nil, errors.WrapError(errors.ErrCSVEncodeFailed, + errors.Errorf("unsupported binary encoding method %s", + csvConfig.BinaryEncodingMethod)) + } + } + val = []byte(str) + case mysql.TypeNewDecimal: + val = types.NewDecFromStringForTest(str) + case mysql.TypeFloat: + val, err = strconv.ParseFloat(str, 32) + case mysql.TypeDouble: + val, err = strconv.ParseFloat(str, 64) + case mysql.TypeTiny, mysql.TypeShort, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong: + if mysql.HasUnsignedFlag(ft.GetFlag()) { + val, err = strconv.ParseUint(str, 10, 64) + } else { + val, err = strconv.ParseInt(str, 10, 64) + } + case mysql.TypeYear: + val, err = strconv.ParseInt(str, 10, 64) + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeTimestamp: + val, err = types.ParseTime(types.DefaultStmtNoWarningContext, str, ft.GetType(), ft.GetDecimal()) + case mysql.TypeDuration: + val, _, err = types.ParseDuration(types.DefaultStmtNoWarningContext, str, ft.GetDecimal()) + case mysql.TypeBit: + val, err = types.NewBitLiteral(str) + case mysql.TypeSet: + val, err = types.ParseSet(ft.GetElems(), str, ft.GetCollate()) + case mysql.TypeEnum: + val, err = types.ParseEnum(ft.GetElems(), str, ft.GetCollate()) + case mysql.TypeJSON: + val, err = types.ParseBinaryJSONFromString(str) + case mysql.TypeTiDBVectorFloat32: + val, err = types.ParseVectorFloat32(str) + default: + return str, nil + } + return val, err +} + +func csvMsg2RowChangedEvent(csvConfig *common.Config, csvMsg *csvMessage, tableInfo *commonType.TableInfo) (*commonEvent.DMLEvent, error) { + var err error + if len(csvMsg.columns) != len(tableInfo.GetColumns()) { + return nil, errors.WrapError(errors.ErrCSVDecodeFailed, + fmt.Errorf("the column length of csv message %d doesn't equal to that of tableInfo %d", + len(csvMsg.columns), len(tableInfo.GetColumns()))) + } + + e := new(commonEvent.DMLEvent) + e.CommitTs = csvMsg.commitTs + e.TableInfo = tableInfo + + chk := chunk.NewChunkWithCapacity(tableInfo.GetFieldSlice(), 1) + columns := tableInfo.GetColumns() + data, err := formatAllColumnsValue(csvConfig, csvMsg.columns, columns) + if err != nil { + return nil, err + } + common.AppendRow2Chunk(data, columns, chk) + return e, nil +} + +func formatAllColumnsValue(csvConfig *common.Config, csvCols []any, ticols []*model.ColumnInfo) (map[string]any, error) { + data := make(map[string]interface{}, 0) + for idx, csvCol := range csvCols { + ticol := ticols[idx] + val, err := fromCsvValToColValue(csvConfig, csvCol, ticol.FieldType) + if err != nil { + return nil, err + } + data[ticol.Name.O] = val + } + + return data, nil +} diff --git a/pkg/sink/codec/csv/csv_decoder_test.go b/pkg/sink/codec/csv/csv_decoder_test.go new file mode 100644 index 0000000000..71ea157cd6 --- /dev/null +++ b/pkg/sink/codec/csv/csv_decoder_test.go @@ -0,0 +1,60 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +package csv + +import ( + "context" + "testing" + + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + "github.com/stretchr/testify/require" +) + +func TestCSVBatchDecoder(t *testing.T) { + csvData := `"I","employee","hr",433305438660591626,101,"Smith","Bob","2014-06-04","New York" +"U","employee","hr",433305438660591627,101,"Smith","Bob","2015-10-08","Los Angeles" +"D","employee","hr",433305438660591629,101,"Smith","Bob","2017-03-13","Dallas" +"I","employee","hr",433305438660591630,102,"Alex","Alice","2017-03-14","Shanghai" +"U","employee","hr",433305438660591630,102,"Alex","Alice","2018-06-15","Beijing" +` + ctx := context.Background() + helper := commonEvent.NewEventTestHelper(t) + defer helper.Close() + + _ = helper.DDL2Job("create database hr") + createTableDDL := helper.DDL2Event("create table hr.employee(Id int, LastName varchar(255), FirstName varchar(255), HireDate date, OfficeLocation varchar(255))") + + codecConfig := &common.Config{ + Delimiter: ",", + Quote: "\"", + Terminator: "\n", + NullString: "\\N", + IncludeCommitTs: true, + } + decoder, err := NewBatchDecoder(ctx, codecConfig, createTableDDL.TableInfo, []byte(csvData)) + require.NoError(t, err) + + for i := 0; i < 5; i++ { + tp, hasNext, err := decoder.HasNext() + require.Nil(t, err) + require.True(t, hasNext) + require.Equal(t, common.MessageTypeRow, tp) + event, err := decoder.NextDMLEvent() + require.NoError(t, err) + require.NotNil(t, event) + } + + _, hasNext, _ := decoder.HasNext() + require.False(t, hasNext) +} diff --git a/pkg/sink/codec/csv/csv_message.go b/pkg/sink/codec/csv/csv_message.go index 7833ad27b1..e1381f63aa 100644 --- a/pkg/sink/codec/csv/csv_message.go +++ b/pkg/sink/codec/csv/csv_message.go @@ -20,7 +20,6 @@ import ( "strconv" "strings" - "github.com/pingcap/errors" "github.com/pingcap/log" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/common/event" @@ -30,7 +29,7 @@ import ( "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/errors" ) // a csv row should at least contain operation-type, table-name, schema-name and one table column @@ -152,13 +151,13 @@ func (c *csvMessage) encodeColumns(columns []any, b *strings.Builder) { func (c *csvMessage) decode(datums []types.Datum) error { var dataColIdx int if len(datums) < minimumColsCnt { - return cerror.WrapError(cerror.ErrCSVDecodeFailed, + return errors.WrapError(errors.ErrCSVDecodeFailed, errors.New("the csv row should have at least four columns"+ "(operation-type, table-name, schema-name, commit-ts)")) } if err := c.opType.FromString(datums[0].GetString()); err != nil { - return cerror.WrapError(cerror.ErrCSVDecodeFailed, err) + return errors.WrapError(errors.ErrCSVDecodeFailed, err) } dataColIdx++ c.tableName = datums[1].GetString() @@ -168,7 +167,7 @@ func (c *csvMessage) decode(datums []types.Datum) error { if c.config.IncludeCommitTs { commitTs, err := strconv.ParseUint(datums[3].GetString(), 10, 64) if err != nil { - return cerror.WrapError(cerror.ErrCSVDecodeFailed, + return errors.WrapError(errors.ErrCSVDecodeFailed, fmt.Errorf("the 4th column(%s) of csv row should be a valid commit-ts", datums[3].GetString())) } c.commitTs = commitTs @@ -291,7 +290,7 @@ func fromColValToCsvVal(csvConfig *common.Config, row *chunk.Row, idx int, colIn case config.BinaryEncodingHex: return hex.EncodeToString(v), nil default: - return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, + return nil, errors.WrapError(errors.ErrCSVEncodeFailed, errors.Errorf("unsupported binary encoding method %s", csvConfig.BinaryEncodingMethod)) } @@ -305,14 +304,14 @@ func fromColValToCsvVal(csvConfig *common.Config, row *chunk.Row, idx int, colIn enumValue := row.GetEnum(idx).Value enumVar, err := types.ParseEnumValue(colInfo.GetElems(), enumValue) if err != nil { - return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, err) + return nil, errors.WrapError(errors.ErrCSVEncodeFailed, err) } return enumVar.Name, nil case mysql.TypeSet: bitValue := row.GetEnum(idx).Value setVar, err := types.ParseSetValue(colInfo.GetElems(), bitValue) if err != nil { - return nil, cerror.WrapError(cerror.ErrCSVEncodeFailed, err) + return nil, errors.WrapError(errors.ErrCSVEncodeFailed, err) } return setVar.Name, nil case mysql.TypeBit: @@ -367,7 +366,7 @@ func rowChangedEvent2CSVMsg(csvConfig *common.Config, e *event.RowEvent) (*csvMe csvMsg.opType = operationUpdate if csvConfig.OutputOldValue { if e.GetPreRows().Len() != e.GetRows().Len() { - return nil, cerror.WrapError(cerror.ErrCSVDecodeFailed, + return nil, errors.WrapError(errors.ErrCSVDecodeFailed, fmt.Errorf("the column length of preColumns %d doesn't equal to that of columns %d", e.GetPreRows().Len(), e.GetRows().Len())) } From 9ff0efb90e4f6852b076e88415492cb3ab8955b3 Mon Sep 17 00:00:00 2001 From: nhsmw Date: Thu, 17 Apr 2025 17:57:54 +0800 Subject: [PATCH 10/13] codec: add debezium protocol decoder (#1234) close pingcap/ticdc#1232 --- pkg/errors/error.go | 8 + pkg/sink/codec/debezium/decode.go | 395 ++++++++++++++++++++++++++++++ 2 files changed, 403 insertions(+) create mode 100644 pkg/sink/codec/debezium/decode.go diff --git a/pkg/errors/error.go b/pkg/errors/error.go index a73711d3e4..b87e290d33 100644 --- a/pkg/errors/error.go +++ b/pkg/errors/error.go @@ -241,6 +241,14 @@ var ( "debezium encode failed", errors.RFCCodeText("CDC:ErrDebeziumEncodeFailed"), ) + ErrDebeziumInvalidMessage = errors.Normalize( + "debezium invalid message format, %s", + errors.RFCCodeText("CDC:ErrDebeziumInvalidMessage"), + ) + ErrDebeziumEmptyValueMessage = errors.Normalize( + "debezium value should not be empty", + errors.RFCCodeText("CDC:ErrDebeziumEmptyValueMessage"), + ) ErrStorageSinkInvalidConfig = errors.Normalize( "storage sink config invalid", errors.RFCCodeText("CDC:ErrStorageSinkInvalidConfig"), diff --git a/pkg/sink/codec/debezium/decode.go b/pkg/sink/codec/debezium/decode.go new file mode 100644 index 0000000000..9cd548cba7 --- /dev/null +++ b/pkg/sink/codec/debezium/decode.go @@ -0,0 +1,395 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package debezium + +import ( + "bytes" + "database/sql" + "encoding/base64" + "encoding/json" + "fmt" + "math" + "strings" + + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + timodel "github.com/pingcap/tidb/pkg/meta/model" + pmodel "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + ptypes "github.com/pingcap/tidb/pkg/parser/types" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "go.uber.org/zap" +) + +// Decoder implement the RowEventDecoder interface +type Decoder struct { + config *common.Config + + upstreamTiDB *sql.DB + tableIDAllocator *common.FakeTableIDAllocator + + keyPayload map[string]interface{} + keySchema map[string]interface{} + valuePayload map[string]interface{} + valueSchema map[string]interface{} +} + +// NewDecoder return an debezium decoder +func NewDecoder( + config *common.Config, + db *sql.DB, +) common.RowEventDecoder { + return &Decoder{ + config: config, + upstreamTiDB: db, + tableIDAllocator: common.NewFakeTableIDAllocator(), + } +} + +// AddKeyValue add the received key and values to the Decoder +func (d *Decoder) AddKeyValue(key, value []byte) error { + if d.valuePayload != nil || d.valueSchema != nil { + return errors.New("key or value is not nil") + } + keyPayload, keySchema, err := decodeRawBytes(key) + if err != nil { + return errors.ErrDebeziumEncodeFailed.FastGenByArgs(err) + } + valuePayload, valueSchema, err := decodeRawBytes(value) + if err != nil { + return errors.ErrDebeziumEncodeFailed.FastGenByArgs(err) + } + d.keyPayload = keyPayload + d.keySchema = keySchema + d.valuePayload = valuePayload + d.valueSchema = valueSchema + return nil +} + +// HasNext returns whether there is any event need to be consumed +func (d *Decoder) HasNext() (common.MessageType, bool, error) { + if d.valuePayload == nil && d.valueSchema == nil { + return common.MessageTypeUnknown, false, nil + } + + if len(d.valuePayload) < 1 { + return common.MessageTypeUnknown, false, errors.ErrDebeziumInvalidMessage.FastGenByArgs(d.valuePayload) + } + op, ok := d.valuePayload["op"] + if !ok { + return common.MessageTypeDDL, true, nil + } + switch op { + case "c", "u", "d": + return common.MessageTypeRow, true, nil + case "m": + return common.MessageTypeResolved, true, nil + } + return common.MessageTypeUnknown, false, errors.ErrDebeziumInvalidMessage.FastGenByArgs(d.valuePayload) +} + +// NextResolvedEvent returns the next resolved event if exists +func (d *Decoder) NextResolvedEvent() (uint64, error) { + if len(d.valuePayload) == 0 { + return 0, errors.ErrDebeziumEmptyValueMessage + } + commitTs := d.getCommitTs() + d.clear() + return commitTs, nil +} + +// NextDDLEvent returns the next DDL event if exists +func (d *Decoder) NextDDLEvent() (*commonEvent.DDLEvent, error) { + if len(d.valuePayload) == 0 { + return nil, errors.ErrDebeziumEmptyValueMessage + } + defer d.clear() + event := new(commonEvent.DDLEvent) + event.TableInfo = new(commonType.TableInfo) + tableName := d.getTableName() + if tableName != "" { + event.TableInfo.TableName = commonType.TableName{ + Schema: d.getSchemaName(), + Table: tableName, + } + } + event.Query = d.valuePayload["ddl"].(string) + event.FinishedTs = d.getCommitTs() + return event, nil +} + +// NextDMLEvent returns the next dml event if exists +func (d *Decoder) NextDMLEvent() (*commonEvent.DMLEvent, error) { + if len(d.valuePayload) == 0 { + return nil, errors.ErrDebeziumEmptyValueMessage + } + if d.config.DebeziumDisableSchema { + return nil, errors.ErrDebeziumInvalidMessage.GenWithStackByArgs("DebeziumDisableSchema is true") + } + if !d.config.EnableTiDBExtension { + return nil, errors.ErrDebeziumInvalidMessage.GenWithStackByArgs("EnableTiDBExtension is false") + } + defer d.clear() + tableInfo := d.getTableInfo() + commitTs := d.getCommitTs() + event := &commonEvent.DMLEvent{ + CommitTs: commitTs, + TableInfo: tableInfo, + } + chk := chunk.NewChunkWithCapacity(tableInfo.GetFieldSlice(), 1) + columns := tableInfo.GetColumns() + if before, ok := d.valuePayload["before"].(map[string]interface{}); ok { + data := assembleColumnData(before, columns) + common.AppendRow2Chunk(data, columns, chk) + } + if after, ok := d.valuePayload["after"].(map[string]interface{}); ok { + data := assembleColumnData(after, columns) + common.AppendRow2Chunk(data, columns, chk) + } + event.PhysicalTableID = d.tableIDAllocator.AllocateTableID(tableInfo.GetSchemaName(), tableInfo.GetTableName()) + return event, nil +} + +func (d *Decoder) getCommitTs() uint64 { + source := d.valuePayload["source"].(map[string]interface{}) + commitTs, err := source["commit_ts"].(json.Number).Int64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", source)) + } + return uint64(commitTs) +} + +func (d *Decoder) getSchemaName() string { + source := d.valuePayload["source"].(map[string]interface{}) + schemaName := source["db"].(string) + return schemaName +} + +func (d *Decoder) getTableName() string { + source := d.valuePayload["source"].(map[string]interface{}) + tableName := source["table"].(string) + return tableName +} + +func (d *Decoder) clear() { + d.keyPayload = nil + d.keySchema = nil + d.valuePayload = nil + d.valueSchema = nil +} + +func (d *Decoder) getTableInfo() *commonType.TableInfo { + tidbTableInfo := new(timodel.TableInfo) + tidbTableInfo.Name = pmodel.NewCIStr(d.getTableName()) + fields := d.valueSchema["fields"].([]interface{}) + after := fields[1].(map[string]interface{}) + columnsField := after["fields"].([]interface{}) + indexColumns := make([]*timodel.IndexColumn, 0, len(d.keyPayload)) + for idx, column := range columnsField { + col := column.(map[string]interface{}) + colName := col["field"].(string) + tidbType := col["tidb_type"].(string) + optional := col["optional"].(bool) + fieldType := parseTiDBType(tidbType, optional) + if fieldType.GetType() == mysql.TypeDatetime { + name := col["name"].(string) + if name == "io.debezium.time.MicroTimestamp" { + fieldType.SetDecimal(6) + } + } + if _, ok := d.keyPayload[colName]; ok { + indexColumns = append(indexColumns, &timodel.IndexColumn{ + Name: pmodel.NewCIStr(colName), + Offset: idx, + }) + fieldType.AddFlag(mysql.PriKeyFlag) + } + tidbTableInfo.Columns = append(tidbTableInfo.Columns, &timodel.ColumnInfo{ + ID: int64(idx), + State: timodel.StatePublic, + Name: pmodel.NewCIStr(colName), + FieldType: *fieldType, + }) + } + tidbTableInfo.Indices = append(tidbTableInfo.Indices, &timodel.IndexInfo{ + ID: 1, + Name: pmodel.NewCIStr("primary"), + Columns: indexColumns, + Unique: true, + Primary: true, + }) + return commonType.NewTableInfo4Decoder(d.getSchemaName(), tidbTableInfo) +} + +func assembleColumnData(data map[string]interface{}, columns []*timodel.ColumnInfo) map[string]interface{} { + result := make(map[string]interface{}, 0) + for _, col := range columns { + val, ok := data[col.Name.O] + if !ok { + continue + } + result[col.Name.O] = decodeColumn(val, col) + } + return result +} + +func decodeColumn(value interface{}, colInfo *timodel.ColumnInfo) interface{} { + if value == nil { + return value + } + var err error + // Notice: value may be the default value of the column + switch colInfo.GetType() { + case mysql.TypeVarchar, mysql.TypeString, mysql.TypeVarString, mysql.TypeTinyBlob, + mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: + if mysql.HasBinaryFlag(colInfo.GetFlag()) { + s := value.(string) + value, err = base64.StdEncoding.DecodeString(s) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + } + case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate: + val := value.(json.Number).String() + value, err = types.ParseTime(types.DefaultStmtNoWarningContext, val, colInfo.GetType(), colInfo.GetDecimal()) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + case mysql.TypeDuration: + val := value.(json.Number).String() + value, _, err := types.ParseDuration(types.DefaultStmtNoWarningContext, val, colInfo.GetDecimal()) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + case mysql.TypeLonglong, mysql.TypeLong, mysql.TypeInt24, mysql.TypeShort, mysql.TypeTiny: + v, err := value.(json.Number).Int64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + if mysql.HasUnsignedFlag(colInfo.GetFlag()) { + value = uint64(v) + } else { + if v > math.MaxInt64 { + value = math.MaxInt64 + } else if v < math.MinInt64 { + value = math.MinInt64 + } else { + value = v + } + } + case mysql.TypeBit: + switch val := value.(type) { + case string: + value, err = types.NewBitLiteral(val) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + case bool: + if val { + value = types.NewBinaryLiteralFromUint(uint64(1), -1) + } else { + value = types.NewBinaryLiteralFromUint(uint64(0), -1) + } + } + case mysql.TypeNewDecimal: + v, err := value.(json.Number).Float64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + value = types.NewDecFromFloatForTest(v) + case mysql.TypeDouble: + value, err := value.(json.Number).Float64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + case mysql.TypeFloat: + v, err := value.(json.Number).Float64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + value = float32(v) + case mysql.TypeYear: + value, err = value.(json.Number).Int64() + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + + } + case mysql.TypeEnum: + value, err = types.ParseEnumName(colInfo.GetElems(), value.(string), colInfo.GetCollate()) + case mysql.TypeSet: + value, err = types.ParseSetName(colInfo.GetElems(), value.(string), colInfo.GetCollate()) + case mysql.TypeJSON: + value, err = types.ParseBinaryJSONFromString(value.(string)) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + case mysql.TypeTiDBVectorFloat32: + value, err = types.ParseVectorFloat32(value.(string)) + if err != nil { + log.Error("decode value failed", zap.Error(err), zap.Any("value", value)) + return nil + } + default: + } + return value +} + +func parseTiDBType(tidbType string, optional bool) *ptypes.FieldType { + ft := new(ptypes.FieldType) + if optional { + ft.AddFlag(mysql.NotNullFlag) + } + if strings.Contains(tidbType, " unsigned") { + ft.AddFlag(mysql.UnsignedFlag) + tidbType = strings.Replace(tidbType, " unsigned", "", 1) + } + if strings.Contains(tidbType, "blob") || strings.Contains(tidbType, "binary") { + ft.AddFlag(mysql.BinaryFlag) + } + tp := ptypes.StrToType(tidbType) + ft.SetType(tp) + return ft +} + +func decodeRawBytes(data []byte) (map[string]interface{}, map[string]interface{}, error) { + var v map[string]interface{} + d := json.NewDecoder(bytes.NewBuffer(data)) + d.UseNumber() + if err := d.Decode(&v); err != nil { + return nil, nil, errors.Trace(err) + } + payload, ok := v["payload"].(map[string]interface{}) + if !ok { + return nil, nil, fmt.Errorf("decode payload failed, data: %+v", v) + } + schema, ok := v["schema"].(map[string]interface{}) + if !ok { + return nil, nil, fmt.Errorf("decode payload failed, data: %+v", v) + } + return payload, schema, nil +} From a800a8c426c0243dd3fee9d01bafe4b070f09f73 Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Fri, 18 Apr 2025 17:24:34 +0800 Subject: [PATCH 11/13] cloudstorage: fix a bug that may cause changefeed getting stuck (#1238) close pingcap/ticdc#1239 --- cmd/storage-consumer/main.go | 3 +- downstreamadapter/sink/cloudstorage.go | 4 +- downstreamadapter/sink/helper/helper.go | 69 ---------- .../worker/cloudstorage_ddl_worker_test.go | 5 +- .../worker/cloudstorage_dml_worker_test.go | 2 +- .../worker/writer/writer_test.go | 3 +- pkg/redo/config.go | 2 +- pkg/sink/cloudstorage/path_test.go | 5 +- pkg/sink/codec/open/encoder_test.go | 3 + pkg/util/external_storage.go | 70 +++++++--- pkg/util/external_storage_test.go | 127 ++++++++++++++++++ pkg/util/tz_test.go | 50 +++++++ 12 files changed, 241 insertions(+), 102 deletions(-) create mode 100644 pkg/util/external_storage_test.go create mode 100644 pkg/util/tz_test.go diff --git a/cmd/storage-consumer/main.go b/cmd/storage-consumer/main.go index 18367705f9..e4573954e7 100644 --- a/cmd/storage-consumer/main.go +++ b/cmd/storage-consumer/main.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/cmd/util" - "github.com/pingcap/ticdc/downstreamadapter/sink/helper" "github.com/pingcap/ticdc/pkg/spanz" putil "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/ticdc/pkg/version" @@ -180,7 +179,7 @@ func newConsumer(ctx context.Context) (*consumer, error) { extension := sinkutil.GetFileExtension(protocol) - storage, err := helper.GetExternalStorageFromURI(ctx, upstreamURIStr) + storage, err := putil.GetExternalStorageWithDefaultTimeout(ctx, upstreamURIStr) if err != nil { log.Error("failed to create external storage", zap.Error(err)) return nil, err diff --git a/downstreamadapter/sink/cloudstorage.go b/downstreamadapter/sink/cloudstorage.go index 299c0822c5..42bc6f7f99 100644 --- a/downstreamadapter/sink/cloudstorage.go +++ b/downstreamadapter/sink/cloudstorage.go @@ -73,7 +73,7 @@ func verifyCloudStorageSink(ctx context.Context, changefeedID common.ChangeFeedI if _, err = util.GetEncoderConfig(changefeedID, sinkURI, protocol, sinkConfig, math.MaxInt); err != nil { return err } - if storage, err = helper.GetExternalStorageFromURI(ctx, sinkURI.String()); err != nil { + if storage, err = putil.GetExternalStorageWithDefaultTimeout(ctx, sinkURI.String()); err != nil { return err } storage.Close() @@ -105,7 +105,7 @@ func newCloudStorageSink( if err != nil { return nil, errors.Trace(err) } - storage, err := helper.GetExternalStorageFromURI(ctx, sinkURI.String()) + storage, err := putil.GetExternalStorageWithDefaultTimeout(ctx, sinkURI.String()) if err != nil { return nil, err } diff --git a/downstreamadapter/sink/helper/helper.go b/downstreamadapter/sink/helper/helper.go index 9bbbae553d..3d33a68261 100644 --- a/downstreamadapter/sink/helper/helper.go +++ b/downstreamadapter/sink/helper/helper.go @@ -14,16 +14,11 @@ package helper import ( - "context" "net/url" "strings" - "time" - "github.com/aws/aws-sdk-go/aws/client" - "github.com/aws/aws-sdk-go/aws/request" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/tidb/br/pkg/storage" ) // DDLDispatchRule is the dispatch rule for DDL event. @@ -46,70 +41,6 @@ func GetDDLDispatchRule(protocol config.Protocol) DDLDispatchRule { return PartitionAll } -// GetExternalStorageFromURI creates a new storage.ExternalStorage from a uri. -func GetExternalStorageFromURI( - ctx context.Context, uri string, -) (storage.ExternalStorage, error) { - return GetExternalStorage(ctx, uri, nil, DefaultS3Retryer()) -} - -// GetExternalStorage creates a new storage.ExternalStorage based on the uri and options. -func GetExternalStorage( - ctx context.Context, uri string, - opts *storage.BackendOptions, - retryer request.Retryer, -) (storage.ExternalStorage, error) { - backEnd, err := storage.ParseBackend(uri, opts) - if err != nil { - return nil, errors.Trace(err) - } - - ret, err := storage.New(ctx, backEnd, &storage.ExternalStorageOptions{ - SendCredentials: false, - S3Retryer: retryer, - }) - if err != nil { - retErr := errors.ErrFailToCreateExternalStorage.Wrap(errors.Trace(err)) - return nil, retErr.GenWithStackByArgs("creating ExternalStorage for s3") - } - - // Check the connection and ignore the returned bool value, since we don't care if the file exists. - _, err = ret.FileExists(ctx, "test") - if err != nil { - retErr := errors.ErrFailToCreateExternalStorage.Wrap(errors.Trace(err)) - return nil, retErr.GenWithStackByArgs("creating ExternalStorage for s3") - } - return ret, nil -} - -// retryerWithLog wraps the client.DefaultRetryer, and logs when retrying. -type retryerWithLog struct { - client.DefaultRetryer -} - -// DefaultS3Retryer is the default s3 retryer, maybe this function -// should be extracted to another place. -func DefaultS3Retryer() request.Retryer { - return retryerWithLog{ - DefaultRetryer: client.DefaultRetryer{ - NumMaxRetries: 3, - MinRetryDelay: 1 * time.Second, - MinThrottleDelay: 2 * time.Second, - }, - } -} - -// NewS3Retryer creates a new s3 retryer. -func NewS3Retryer(maxRetries int, minRetryDelay, minThrottleDelay time.Duration) request.Retryer { - return retryerWithLog{ - DefaultRetryer: client.DefaultRetryer{ - NumMaxRetries: maxRetries, - MinRetryDelay: minRetryDelay, - MinThrottleDelay: minThrottleDelay, - }, - } -} - // GetTopic returns the topic name from the sink URI. func GetTopic(sinkURI *url.URL) (string, error) { topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { diff --git a/downstreamadapter/worker/cloudstorage_ddl_worker_test.go b/downstreamadapter/worker/cloudstorage_ddl_worker_test.go index a26bc2232b..fc9a57bc36 100644 --- a/downstreamadapter/worker/cloudstorage_ddl_worker_test.go +++ b/downstreamadapter/worker/cloudstorage_ddl_worker_test.go @@ -23,7 +23,6 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/downstreamadapter/sink/helper" "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" @@ -59,7 +58,7 @@ func newCloudStorageDDLWorkerForTest(parentDir string) (*CloudStorageDDLWorker, if err != nil { return nil, err } - storage, err := helper.GetExternalStorageFromURI(ctx, sinkURI.String()) + storage, err := util.GetExternalStorageWithDefaultTimeout(ctx, sinkURI.String()) if err != nil { return nil, err } @@ -162,7 +161,7 @@ func TestCleanupExpiredFiles(t *testing.T) { cfg := cloudstorage.NewConfig() err = cfg.Apply(ctx, sinkURI, replicaConfig.Sink) require.Nil(t, err) - storage, err := helper.GetExternalStorageFromURI(ctx, sinkURI.String()) + storage, err := util.GetExternalStorageWithDefaultTimeout(ctx, sinkURI.String()) require.Nil(t, err) sink := NewCloudStorageDDLWorker(changefeedID, sinkURI, cfg, cleanupJobs, storage, metrics.NewStatistics(changefeedID, "CloudStorageSink")) diff --git a/downstreamadapter/worker/cloudstorage_dml_worker_test.go b/downstreamadapter/worker/cloudstorage_dml_worker_test.go index 406d0049a0..205d35cacf 100644 --- a/downstreamadapter/worker/cloudstorage_dml_worker_test.go +++ b/downstreamadapter/worker/cloudstorage_dml_worker_test.go @@ -98,7 +98,7 @@ func newCloudStorageDMLWorkerForTest(parentDir string, flushInterval int, sinkCo if err != nil { return nil, errors.Trace(err) } - storage, err := helper.GetExternalStorageFromURI(ctx, sinkURI.String()) + storage, err := putil.GetExternalStorageWithDefaultTimeout(ctx, sinkURI.String()) if err != nil { return nil, err } diff --git a/downstreamadapter/worker/writer/writer_test.go b/downstreamadapter/worker/writer/writer_test.go index 8b3e8f8333..f471386743 100644 --- a/downstreamadapter/worker/writer/writer_test.go +++ b/downstreamadapter/worker/writer/writer_test.go @@ -22,7 +22,6 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/downstreamadapter/sink/helper" commonType "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" commonEvent "github.com/pingcap/ticdc/pkg/common/event" @@ -62,7 +61,7 @@ func getTableFiles(t *testing.T, tableDir string) []string { func testWriter(ctx context.Context, t *testing.T, dir string) *Writer { uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) - storage, err := helper.GetExternalStorageFromURI(ctx, uri) + storage, err := util.GetExternalStorageWithDefaultTimeout(ctx, uri) require.Nil(t, err) sinkURI, err := url.Parse(uri) require.Nil(t, err) diff --git a/pkg/redo/config.go b/pkg/redo/config.go index daef33354d..a20c8c1e38 100644 --- a/pkg/redo/config.go +++ b/pkg/redo/config.go @@ -205,7 +205,7 @@ func initExternalStorageForTest(ctx context.Context, uri url.URL) (storage.Exter return nil, errors.WrapError(errors.ErrStorageInitialize, errors.Errorf("please specify the bucket for %+v", uri)) } - s, err := util.GetExternalStorageFromURI(ctx, uri.String()) + s, err := util.GetExternalStorageWithDefaultTimeout(ctx, uri.String()) if err != nil { return nil, errors.WrapError(errors.ErrStorageInitialize, err) } diff --git a/pkg/sink/cloudstorage/path_test.go b/pkg/sink/cloudstorage/path_test.go index 5b39d1212b..9173645365 100644 --- a/pkg/sink/cloudstorage/path_test.go +++ b/pkg/sink/cloudstorage/path_test.go @@ -23,7 +23,6 @@ import ( "time" "github.com/google/uuid" - "github.com/pingcap/ticdc/downstreamadapter/sink/helper" commonType "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/config" @@ -40,7 +39,7 @@ import ( func testFilePathGenerator(ctx context.Context, t *testing.T, dir string) *FilePathGenerator { uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) - storage, err := helper.GetExternalStorageFromURI(ctx, uri) + storage, err := util.GetExternalStorageWithDefaultTimeout(ctx, uri) require.NoError(t, err) sinkURI, err := url.Parse(uri) @@ -361,7 +360,7 @@ func TestRemoveExpiredFilesWithoutPartition(t *testing.T) { defer cancel() dir := t.TempDir() uri := fmt.Sprintf("file:///%s?flush-interval=2s", dir) - storage, err := helper.GetExternalStorageFromURI(ctx, uri) + storage, err := util.GetExternalStorageWithDefaultTimeout(ctx, uri) require.NoError(t, err) sinkURI, err := url.Parse(uri) require.NoError(t, err) diff --git a/pkg/sink/codec/open/encoder_test.go b/pkg/sink/codec/open/encoder_test.go index 7f1de5a5b3..5c3a4731fb 100644 --- a/pkg/sink/codec/open/encoder_test.go +++ b/pkg/sink/codec/open/encoder_test.go @@ -17,6 +17,7 @@ import ( "context" "testing" + "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common/columnselector" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" @@ -1126,4 +1127,6 @@ func TestHandleOnlyEvent(t *testing.T) { require.True(t, ok) common.CompareRow(t, insertRowEvent.Event, insertRowEvent.TableInfo, change, decoded.TableInfo) + + log.Info("pass TestHandleOnlyEvent") } diff --git a/pkg/util/external_storage.go b/pkg/util/external_storage.go index c557cde317..39f77474d1 100644 --- a/pkg/util/external_storage.go +++ b/pkg/util/external_storage.go @@ -37,13 +37,6 @@ import ( const defaultTimeout = 5 * time.Minute -// GetExternalStorageFromURI creates a new storage.ExternalStorage from a uri. -func GetExternalStorageFromURI( - ctx context.Context, uri string, -) (storage.ExternalStorage, error) { - return GetExternalStorage(ctx, uri, nil, DefaultS3Retryer()) -} - // GetExternalStorageWithDefaultTimeout creates a new storage.ExternalStorage from a uri // without retry. It is the caller's responsibility to set timeout to the context. func GetExternalStorageWithDefaultTimeout(ctx context.Context, uri string) (storage.ExternalStorage, error) { @@ -51,7 +44,7 @@ func GetExternalStorageWithDefaultTimeout(ctx context.Context, uri string) (stor defer cancel() // total retry time is [1<<7, 1<<8] = [128, 256] + 30*6 = [308, 436] seconds r := NewS3Retryer(7, 1*time.Second, 2*time.Second) - s, err := GetExternalStorage(ctx, uri, nil, r) + s, err := getExternalStorage(ctx, uri, nil, r) return &extStorageWithTimeout{ ExternalStorage: s, @@ -59,8 +52,8 @@ func GetExternalStorageWithDefaultTimeout(ctx context.Context, uri string) (stor }, err } -// GetExternalStorage creates a new storage.ExternalStorage based on the uri and options. -func GetExternalStorage( +// getExternalStorage creates a new storage.ExternalStorage based on the uri and options. +func getExternalStorage( ctx context.Context, uri string, opts *storage.BackendOptions, retryer request.Retryer, @@ -88,12 +81,19 @@ func GetExternalStorage( return ret, nil } +// getExternalStorageFromURI creates a new storage.ExternalStorage from a uri. +func getExternalStorageFromURI( + ctx context.Context, uri string, +) (storage.ExternalStorage, error) { + return getExternalStorage(ctx, uri, nil, DefaultS3Retryer()) +} + // GetTestExtStorage creates a test storage.ExternalStorage from a uri. func GetTestExtStorage( ctx context.Context, tmpDir string, ) (storage.ExternalStorage, *url.URL, error) { uriStr := fmt.Sprintf("file://%s", tmpDir) - ret, err := GetExternalStorageFromURI(ctx, uriStr) + ret, err := getExternalStorageFromURI(ctx, uriStr) if err != nil { return nil, nil, err } @@ -163,28 +163,44 @@ type extStorageWithTimeout struct { func (s *extStorageWithTimeout) WriteFile(ctx context.Context, name string, data []byte) error { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.WriteFile(ctx, name, data) + err := s.ExternalStorage.WriteFile(ctx, name, data) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("WriteFile") + } + return err } // ReadFile reads a complete file from storage, similar to os.ReadFile func (s *extStorageWithTimeout) ReadFile(ctx context.Context, name string) ([]byte, error) { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.ReadFile(ctx, name) + data, err := s.ExternalStorage.ReadFile(ctx, name) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("ReadFile") + } + return data, err } // FileExists return true if file exists func (s *extStorageWithTimeout) FileExists(ctx context.Context, name string) (bool, error) { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.FileExists(ctx, name) + exists, err := s.ExternalStorage.FileExists(ctx, name) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("FileExists") + } + return exists, err } // DeleteFile delete the file in storage func (s *extStorageWithTimeout) DeleteFile(ctx context.Context, name string) error { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.DeleteFile(ctx, name) + err := s.ExternalStorage.DeleteFile(ctx, name) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("DeleteFile") + } + return err } // Open a Reader by file path. path is relative path to storage base path @@ -193,7 +209,11 @@ func (s *extStorageWithTimeout) Open( ) (storage.ExternalFileReader, error) { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.Open(ctx, path, nil) + reader, err := s.ExternalStorage.Open(ctx, path, nil) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("Open") + } + return reader, err } // WalkDir traverse all the files in a dir. @@ -202,7 +222,11 @@ func (s *extStorageWithTimeout) WalkDir( ) error { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.WalkDir(ctx, opt, fn) + err := s.ExternalStorage.WalkDir(ctx, opt, fn) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("WalkDir") + } + return err } // Create opens a file writer by path. path is relative path to storage base path @@ -215,7 +239,11 @@ func (s *extStorageWithTimeout) Create( defer cancel() } // multipart uploading spawns a background goroutine, can't set timeout - return s.ExternalStorage.Create(ctx, path, option) + writer, err := s.ExternalStorage.Create(ctx, path, option) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("Create") + } + return writer, err } // Rename file name from oldFileName to newFileName @@ -224,7 +252,11 @@ func (s *extStorageWithTimeout) Rename( ) error { ctx, cancel := context.WithTimeout(ctx, s.timeout) defer cancel() - return s.ExternalStorage.Rename(ctx, oldFileName, newFileName) + err := s.ExternalStorage.Rename(ctx, oldFileName, newFileName) + if err != nil { + err = errors.ErrExternalStorageAPI.Wrap(err).GenWithStackByArgs("Rename") + } + return err } // IsNotExistInExtStorage checks if the error is caused by the file not exist in external storage. diff --git a/pkg/util/external_storage_test.go b/pkg/util/external_storage_test.go new file mode 100644 index 0000000000..39f6b6fc8e --- /dev/null +++ b/pkg/util/external_storage_test.go @@ -0,0 +1,127 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "context" + "errors" + "net/http" + "testing" + "time" + + "github.com/pingcap/tidb/br/pkg/storage" + "github.com/stretchr/testify/require" +) + +// mockRoundTripper blocks until the context is done. +type mockRoundTripper struct { + blockUntilContextDone bool + err error +} + +func (m *mockRoundTripper) RoundTrip(req *http.Request) (*http.Response, error) { + if m.blockUntilContextDone { + select { + case <-req.Context().Done(): + return nil, req.Context().Err() + } + } + // Return immediately for success case + return &http.Response{StatusCode: http.StatusOK, Body: http.NoBody}, m.err +} + +// mockExternalStorage is a mock implementation for testing timeouts via http client. +type mockExternalStorage struct { + storage.ExternalStorage // Embed the interface to satisfy it easily + httpClient *http.Client +} + +// WriteFile simulates a write operation by making an HTTP request that respects context cancellation. +func (m *mockExternalStorage) WriteFile(ctx context.Context, name string, data []byte) error { + if m.httpClient == nil { + panic("httpClient not set in mockExternalStorage") // Should be set in tests + } + // Create a dummy request. The URL doesn't matter as the RoundTripper is mocked. + req, err := http.NewRequestWithContext(ctx, http.MethodPut, "http://mock/"+name, http.NoBody) + if err != nil { + return err // Should not happen with valid inputs + } + + resp, err := m.httpClient.Do(req) + if err != nil { + return err // This will include context errors like DeadlineExceeded + } + resp.Body.Close() // Important to close the body + if resp.StatusCode != http.StatusOK { + return errors.New("mock http request failed") // Or handle specific statuses + } + return nil +} + +func TestExtStorageWithTimeoutWriteFileTimeout(t *testing.T) { + testTimeout := 50 * time.Millisecond + + // Create a mock HTTP client that blocks until context is done + mockClient := &http.Client{ + Transport: &mockRoundTripper{blockUntilContextDone: true}, + } + + mockStore := &mockExternalStorage{ + httpClient: mockClient, + } + + // Wrap the mock store with the timeout logic + timedStore := &extStorageWithTimeout{ + ExternalStorage: mockStore, + timeout: testTimeout, + } + + startTime := time.Now() + // Use context.Background() as the base context + err := timedStore.WriteFile(context.Background(), "testfile", []byte("data")) + duration := time.Since(startTime) + + // 1. Assert that an error occurred + require.Error(t, err, "Expected an error due to timeout") + + // 2. Assert that the error is context.DeadlineExceeded + require.True(t, errors.Is(err, context.DeadlineExceeded), "Expected context.DeadlineExceeded error, got: %v", err) + + // 3. Assert that the function returned quickly (around the timeout duration) + require.InDelta(t, testTimeout, duration, float64(testTimeout)*0.5, "Duration (%v) should be close to the timeout (%v)", duration, testTimeout) +} + +func TestExtStorageWithTimeoutWriteFileSuccess(t *testing.T) { + testTimeout := 100 * time.Millisecond + + // Create a mock HTTP client that returns success immediately + mockClient := &http.Client{ + Transport: &mockRoundTripper{blockUntilContextDone: false, err: nil}, + } + + mockStore := &mockExternalStorage{ + httpClient: mockClient, + } + + timedStore := &extStorageWithTimeout{ + ExternalStorage: mockStore, + timeout: testTimeout, + } + + // Use context.Background() as the base context + err := timedStore.WriteFile(context.Background(), "testfile", []byte("data")) + + // Assert success + require.NoError(t, err, "Expected no error for successful write within timeout") +} diff --git a/pkg/util/tz_test.go b/pkg/util/tz_test.go new file mode 100644 index 0000000000..39bea01df7 --- /dev/null +++ b/pkg/util/tz_test.go @@ -0,0 +1,50 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestGetTimezoneFromZonefile(t *testing.T) { + testCases := []struct { + hasErr bool + zonefile string + name string + }{ + {true, "", ""}, + {false, "UTC", "UTC"}, + {false, "/usr/share/zoneinfo/UTC", "UTC"}, + {false, "/usr/share/zoneinfo/Etc/UTC", "Etc/UTC"}, + {false, "/usr/share/zoneinfo/Asia/Shanghai", "Asia/Shanghai"}, + } + for _, tc := range testCases { + loc, err := GetTimezoneFromZonefile(tc.zonefile) + if tc.hasErr { + require.NotNil(t, err) + } else { + require.Nil(t, err) + require.Equal(t, tc.name, loc.String()) + } + } +} + +func TestGetTimezoneName(t *testing.T) { + tz, err := GetTimezone("") + require.NoError(t, err) + require.True(t, len(GetTimeZoneName(tz)) != 0) + require.True(t, len(GetTimeZoneName(nil)) == 0) +} From 3c06e2e47cafe2f098277c3782245deb5238adbb Mon Sep 17 00:00:00 2001 From: nhsmw Date: Sat, 19 Apr 2025 01:59:40 +0800 Subject: [PATCH 12/13] sink: send bootstrap event when changefeed with simple protocol start (#1210) ref pingcap/ticdc#1189 --- downstreamadapter/dispatcher/dispatcher.go | 76 +++++++++++++++++++ downstreamadapter/dispatcher/helper.go | 18 +++++ .../event_dispatcher_manager.go | 9 +++ .../persist_storage_ddl_handlers.go | 4 +- maintainer/maintainer_controller.go | 10 +-- maintainer/maintainer_controller_test.go | 4 +- pkg/common/table_info.go | 32 ++++++-- pkg/sink/util/helper.go | 7 ++ .../kafka_simple_basic/run.sh | 5 ++ 9 files changed, 146 insertions(+), 19 deletions(-) diff --git a/downstreamadapter/dispatcher/dispatcher.go b/downstreamadapter/dispatcher/dispatcher.go index 7de71a62c7..fa75715253 100644 --- a/downstreamadapter/dispatcher/dispatcher.go +++ b/downstreamadapter/dispatcher/dispatcher.go @@ -24,12 +24,16 @@ import ( "github.com/pingcap/ticdc/downstreamadapter/syncpoint" "github.com/pingcap/ticdc/eventpb" "github.com/pingcap/ticdc/heartbeatpb" + "github.com/pingcap/ticdc/logservice/schemastore" "github.com/pingcap/ticdc/pkg/apperror" "github.com/pingcap/ticdc/pkg/common" + appcontext "github.com/pingcap/ticdc/pkg/common/context" commonEvent "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/sink/codec" "github.com/pingcap/ticdc/pkg/sink/util" "github.com/pingcap/ticdc/pkg/spanz" "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) @@ -147,6 +151,8 @@ type Dispatcher struct { errCh chan error bdrMode bool + + BootstrapState bootstrapState } func NewDispatcher( @@ -186,6 +192,7 @@ func NewDispatcher( creationPDTs: currentPdTs, errCh: errCh, bdrMode: bdrMode, + BootstrapState: BootstrapFinished, } dispatcher.addToStatusDynamicStream() @@ -801,6 +808,75 @@ func (d *Dispatcher) HandleCheckpointTs(checkpointTs uint64) { d.sink.AddCheckpointTs(checkpointTs) } +// EmitBootstrap emits the table bootstrap event in a blocking way after changefeed started +// It will return after the bootstrap event is sent. +func (d *Dispatcher) EmitBootstrap() bool { + bootstrap := loadBootstrapState(&d.BootstrapState) + switch bootstrap { + case BootstrapFinished: + return true + case BootstrapInProgress: + return false + case BootstrapNotStarted: + } + storeBootstrapState(&d.BootstrapState, BootstrapInProgress) + tables := d.tableSchemaStore.GetAllNormalTableIds() + if len(tables) == 0 { + storeBootstrapState(&d.BootstrapState, BootstrapFinished) + return true + } + start := time.Now() + ts := d.GetStartTs() + schemaStore := appcontext.GetService[schemastore.SchemaStore](appcontext.SchemaStore) + currentTables := make([]*common.TableInfo, 0, len(tables)) + for i := 0; i < len(tables); i++ { + err := schemaStore.RegisterTable(tables[i], ts) + if err != nil { + log.Warn("register table to schemaStore failed", + zap.Int64("tableID", tables[i]), + zap.Uint64("startTs", ts), + zap.Error(err), + ) + continue + } + tableInfo, err := schemaStore.GetTableInfo(tables[i], ts) + if err != nil { + log.Warn("get table info failed, just ignore", + zap.Stringer("changefeed", d.changefeedID), + zap.Error(err)) + continue + } + currentTables = append(currentTables, tableInfo) + } + + log.Info("start to send bootstrap messages", + zap.Stringer("changefeed", d.changefeedID), + zap.Int("tables", len(currentTables))) + for idx, table := range currentTables { + if table.IsView() { + continue + } + ddlEvent := codec.NewBootstrapDDLEvent(table) + err := d.sink.WriteBlockEvent(ddlEvent) + if err != nil { + log.Error("send bootstrap message failed", + zap.Stringer("changefeed", d.changefeedID), + zap.Int("tables", len(currentTables)), + zap.Int("emitted", idx+1), + zap.Duration("duration", time.Since(start)), + zap.Error(err)) + d.errCh <- errors.ErrExecDDLFailed.GenWithStackByArgs() + return true + } + } + storeBootstrapState(&d.BootstrapState, BootstrapFinished) + log.Info("send bootstrap messages finished", + zap.Stringer("changefeed", d.changefeedID), + zap.Int("tables", len(currentTables)), + zap.Duration("cost", time.Since(start))) + return true +} + func (d *Dispatcher) IsTableTriggerEventDispatcher() bool { return d.tableSpan == heartbeatpb.DDLSpan } diff --git a/downstreamadapter/dispatcher/helper.go b/downstreamadapter/dispatcher/helper.go index d7de0e0870..2dac201557 100644 --- a/downstreamadapter/dispatcher/helper.go +++ b/downstreamadapter/dispatcher/helper.go @@ -15,6 +15,7 @@ package dispatcher import ( "sync" + "sync/atomic" "time" "github.com/pingcap/log" @@ -381,3 +382,20 @@ func GetDispatcherStatusDynamicStream() dynstream.DynamicStream[common.GID, comm }) return dispatcherStatusDS } + +// bootstrapState used to check if send bootstrap event after changefeed created +type bootstrapState int32 + +const ( + BootstrapNotStarted bootstrapState = iota + BootstrapInProgress + BootstrapFinished +) + +func storeBootstrapState(addr *bootstrapState, state bootstrapState) { + atomic.StoreInt32((*int32)(addr), int32(state)) +} + +func loadBootstrapState(addr *bootstrapState) bootstrapState { + return bootstrapState(atomic.LoadInt32((*int32)(addr))) +} diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 67a385a835..4e398e0879 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" "github.com/pingcap/ticdc/pkg/pdutil" + "github.com/pingcap/ticdc/pkg/util" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" @@ -417,6 +418,11 @@ func (e *EventDispatcherManager) InitalizeTableTriggerEventDispatcher(schemaInfo if !needAddDispatcher { return nil } + // before bootstrap finished, cannot send any event. + success := e.tableTriggerEventDispatcher.EmitBootstrap() + if !success { + return errors.ErrDispatcherFailed.GenWithStackByArgs() + } // table trigger event dispatcher can register to event collector to receive events after finish the initial table schema store from the maintainer. appcontext.GetService[*eventcollector.EventCollector](appcontext.EventCollector).AddDispatcher(e.tableTriggerEventDispatcher, e.config.MemoryQuota, e.config.BDRMode) @@ -500,6 +506,9 @@ func (e *EventDispatcherManager) newDispatchers(infos []dispatcherCreateInfo, re } if d.IsTableTriggerEventDispatcher() { + if util.GetOrZero(e.config.SinkConfig.SendAllBootstrapAtStart) { + d.BootstrapState = dispatcher.BootstrapNotStarted + } e.tableTriggerEventDispatcher = d } else { e.schemaIDToDispatchers.Set(schemaIds[idx], id) diff --git a/logservice/schemastore/persist_storage_ddl_handlers.go b/logservice/schemastore/persist_storage_ddl_handlers.go index 3dd3a3d0bf..e255548318 100644 --- a/logservice/schemastore/persist_storage_ddl_handlers.go +++ b/logservice/schemastore/persist_storage_ddl_handlers.go @@ -1335,9 +1335,7 @@ func extractTableInfoFuncForExchangeTablePartition(event *PersistedDDLEvent, tab tableID, false, columnSchema, - event.TableInfo.Charset, - event.TableInfo.Collate, - event.TableInfo.Comment, + event.TableInfo, ) return tableInfo, false } diff --git a/maintainer/maintainer_controller.go b/maintainer/maintainer_controller.go index 3ef81c3db1..2a0d7ddc84 100644 --- a/maintainer/maintainer_controller.go +++ b/maintainer/maintainer_controller.go @@ -593,9 +593,8 @@ func (c *Controller) Stop() { func getSchemaInfo(table commonEvent.Table, isMysqlCompatibleBackend bool) *heartbeatpb.SchemaInfo { schemaInfo := &heartbeatpb.SchemaInfo{} - if isMysqlCompatibleBackend { - schemaInfo.SchemaID = table.SchemaID - } else { + schemaInfo.SchemaID = table.SchemaID + if !isMysqlCompatibleBackend { schemaInfo.SchemaName = table.SchemaName } return schemaInfo @@ -603,9 +602,8 @@ func getSchemaInfo(table commonEvent.Table, isMysqlCompatibleBackend bool) *hear func getTableInfo(table commonEvent.Table, isMysqlCompatibleBackend bool) *heartbeatpb.TableInfo { tableInfo := &heartbeatpb.TableInfo{} - if isMysqlCompatibleBackend { - tableInfo.TableID = table.TableID - } else { + tableInfo.TableID = table.TableID + if !isMysqlCompatibleBackend { tableInfo.TableName = table.TableName } return tableInfo diff --git a/maintainer/maintainer_controller_test.go b/maintainer/maintainer_controller_test.go index 679b2d00cc..7ebea04371 100644 --- a/maintainer/maintainer_controller_test.go +++ b/maintainer/maintainer_controller_test.go @@ -408,11 +408,11 @@ func TestFinishBootstrap(t *testing.T) { require.True(t, s.bootstrapped) require.Equal(t, msg.GetSchemas(), []*heartbeatpb.SchemaInfo{ { - // SchemaID: 1, + SchemaID: 1, SchemaName: "test", Tables: []*heartbeatpb.TableInfo{ { - // TableID: 1, + TableID: 1, TableName: "t", }, }, diff --git a/pkg/common/table_info.go b/pkg/common/table_info.go index 7eb3c5075b..00d635445d 100644 --- a/pkg/common/table_info.go +++ b/pkg/common/table_info.go @@ -82,6 +82,10 @@ type TableInfo struct { columnSchema *columnSchema `json:"-"` + View *model.ViewInfo `json:"view"` + + Sequence *model.SequenceInfo `json:"sequence"` + preSQLs struct { mutex sync.Mutex isInitialized bool @@ -287,6 +291,16 @@ func (ti *TableInfo) IsPartitionTable() bool { return ti.TableName.IsPartition } +// IsView checks if TableInfo is a view. +func (t *TableInfo) IsView() bool { + return t.View != nil +} + +// IsSequence checks if TableInfo is a sequence. +func (t *TableInfo) IsSequence() bool { + return t.Sequence != nil +} + // GetRowColInfos returns all column infos for rowcodec func (ti *TableInfo) GetRowColInfos() ([]int64, map[int64]*types.FieldType, []rowcodec.ColInfo) { return ti.columnSchema.HandleColID, ti.columnSchema.RowColFieldTps, ti.columnSchema.RowColInfos @@ -402,7 +416,7 @@ func (ti *TableInfo) IsHandleKey(colID int64) bool { return ok } -func newTableInfo(schema, table string, tableID int64, isPartition bool, columnSchema *columnSchema) *TableInfo { +func newTableInfo(schema string, table string, tableID int64, isPartition bool, columnSchema *columnSchema, tableInfo *model.TableInfo) *TableInfo { ti := &TableInfo{ TableName: TableName{ Schema: schema, @@ -412,15 +426,17 @@ func newTableInfo(schema, table string, tableID int64, isPartition bool, columnS quotedName: QuoteSchema(schema, table), }, columnSchema: columnSchema, + View: tableInfo.View, + Sequence: tableInfo.Sequence, + Charset: tableInfo.Charset, + Collate: tableInfo.Collate, + Comment: tableInfo.Comment, } return ti } -func NewTableInfo(schemaName string, tableName string, tableID int64, isPartition bool, columnSchema *columnSchema, charset, collate, comment string) *TableInfo { - ti := newTableInfo(schemaName, tableName, tableID, isPartition, columnSchema) - ti.Charset = charset - ti.Collate = collate - ti.Comment = comment +func NewTableInfo(schemaName string, tableName string, tableID int64, isPartition bool, columnSchema *columnSchema, tableInfo *model.TableInfo) *TableInfo { + ti := newTableInfo(schemaName, tableName, tableID, isPartition, columnSchema, tableInfo) // when this tableInfo is released, we need to cut down the reference count of the columnSchema // This function should be appeared when tableInfo is created as a pair. @@ -436,14 +452,14 @@ func WrapTableInfo(schemaName string, info *model.TableInfo) *TableInfo { // search column schema object sharedColumnSchemaStorage := GetSharedColumnSchemaStorage() columnSchema := sharedColumnSchemaStorage.GetOrSetColumnSchema(info) - return NewTableInfo(schemaName, info.Name.O, info.ID, info.GetPartitionInfo() != nil, columnSchema, info.Charset, info.Collate, info.Comment) + return NewTableInfo(schemaName, info.Name.O, info.ID, info.GetPartitionInfo() != nil, columnSchema, info) } // NewTableInfo4Decoder is only used by the codec decoder for the test purpose, // do not call this method on the production code. func NewTableInfo4Decoder(schema string, tableInfo *model.TableInfo) *TableInfo { cs := newColumnSchema4Decoder(tableInfo) - return newTableInfo(schema, tableInfo.Name.O, tableInfo.ID, tableInfo.GetPartitionInfo() != nil, cs) + return newTableInfo(schema, tableInfo.Name.O, tableInfo.ID, tableInfo.GetPartitionInfo() != nil, cs, tableInfo) } // BuildTiDBTableInfoWithoutVirtualColumns build a TableInfo without virual columns from the source table info diff --git a/pkg/sink/util/helper.go b/pkg/sink/util/helper.go index 55b5904ed0..8c10a74ea5 100644 --- a/pkg/sink/util/helper.go +++ b/pkg/sink/util/helper.go @@ -96,6 +96,10 @@ func NewTableSchemaStore(schemaInfo []*heartbeatpb.SchemaInfo, sinkType commonTy } else { tableSchemaStore = &TableSchemaStore{ sinkType: sinkType, + tableIDStore: &TableIDStore{ + schemaIDToTableIDs: make(map[int64]map[int64]interface{}), + tableIDToSchemaID: make(map[int64]int64), + }, tableNameStore: &TableNameStore{ existingTables: make(map[string]map[string]*commonEvent.SchemaTableName), latestTableNameChanges: &LatestTableNameChanges{m: make(map[uint64]*commonEvent.TableNameChange)}, @@ -103,9 +107,12 @@ func NewTableSchemaStore(schemaInfo []*heartbeatpb.SchemaInfo, sinkType commonTy } for _, schema := range schemaInfo { schemaName := schema.SchemaName + schemaID := schema.SchemaID for _, table := range schema.Tables { tableName := table.TableName tableSchemaStore.tableNameStore.Add(schemaName, tableName) + tableID := table.TableID + tableSchemaStore.tableIDStore.Add(schemaID, tableID) } } diff --git a/tests/integration_tests/kafka_simple_basic/run.sh b/tests/integration_tests/kafka_simple_basic/run.sh index 23e1b40a04..c8cdb2636a 100644 --- a/tests/integration_tests/kafka_simple_basic/run.sh +++ b/tests/integration_tests/kafka_simple_basic/run.sh @@ -44,8 +44,13 @@ function run() { # pause and resume changefeed makes sure changefeed sending bootstrap events # when it is resumed, so the data after pause can be decoded correctly + TOPIC_NAME="ticdc-simple-basic-$RANDOM" + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=simple" run_cdc_cli changefeed pause -c "simple-basic" + run_cdc_cli changefeed update -c "simple-basic" --sink-uri=$SINK_URI --config="$CUR/conf/changefeed.toml" --no-confirm run_cdc_cli changefeed resume -c "simple-basic" + cdc_kafka_consumer --upstream-uri $SINK_URI --downstream-uri="mysql://root@127.0.0.1:3306/?safe-mode=true&batch-dml-enable=false" --upstream-tidb-dsn="root@tcp(${UP_TIDB_HOST}:${UP_TIDB_PORT})/?" \ + --config="$CUR/conf/changefeed.toml" --log-file $WORK_DIR/cdc_kafka_consumer_resume.log 2>&1 & run_sql_file $CUR/data/data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} From 47cbda302ab7aba736a9ac091c224a364701d532 Mon Sep 17 00:00:00 2001 From: nhsmw Date: Tue, 22 Apr 2025 11:20:38 +0800 Subject: [PATCH 13/13] *: support row checksum (#1245) close pingcap/ticdc#1244 --- api/v2/model.go | 2 +- downstreamadapter/sink/kafka/sink.go | 1 + downstreamadapter/sink/pulsar/sink.go | 1 + eventpb/event.pb.go | 473 +++++++++++++++++++---- eventpb/event.proto | 7 + logservice/eventstore/event_store.go | 3 - pkg/common/event/checksum.go | 175 +++++++++ pkg/common/event/chunk.go | 8 +- pkg/common/event/dml_event.go | 10 +- pkg/common/event/mounter.go | 76 +++- pkg/common/event/row_change.go | 4 +- pkg/common/event/util.go | 3 +- pkg/config/replica_config.go | 2 +- pkg/eventservice/event_broker.go | 4 +- pkg/eventservice/event_broker_test.go | 6 +- pkg/eventservice/event_service.go | 7 +- pkg/eventservice/event_service_test.go | 16 +- pkg/integrity/checksum.go | 22 ++ pkg/integrity/integrity.go | 71 ++++ pkg/messaging/message.go | 21 + pkg/sink/codec/avro/arvo.go | 11 +- pkg/sink/codec/avro/decoder.go | 12 +- pkg/sink/codec/common/helper.go | 30 +- pkg/sink/codec/common/utils.go | 3 + pkg/sink/codec/common/verify_checksum.go | 158 ++------ pkg/sink/codec/debezium/debezium_test.go | 2 +- pkg/sink/codec/simple/avro.go | 25 +- pkg/sink/codec/simple/decoder.go | 34 +- pkg/sink/codec/simple/message.go | 17 +- 29 files changed, 920 insertions(+), 284 deletions(-) create mode 100644 pkg/common/event/checksum.go create mode 100644 pkg/integrity/checksum.go create mode 100644 pkg/integrity/integrity.go diff --git a/api/v2/model.go b/api/v2/model.go index d38adad904..f117b63189 100644 --- a/api/v2/model.go +++ b/api/v2/model.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tiflow/cdc/model" bf "github.com/pingcap/tiflow/pkg/binlog-filter" - "github.com/pingcap/tiflow/pkg/integrity" "github.com/pingcap/tiflow/pkg/security" ) diff --git a/downstreamadapter/sink/kafka/sink.go b/downstreamadapter/sink/kafka/sink.go index 3deb18eb63..bd3a4c6209 100644 --- a/downstreamadapter/sink/kafka/sink.go +++ b/downstreamadapter/sink/kafka/sink.go @@ -247,6 +247,7 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error { Event: row, Callback: rowCallback, ColumnSelector: selector, + Checksum: event.Checksum, }, } s.rowChan <- mqEvent diff --git a/downstreamadapter/sink/pulsar/sink.go b/downstreamadapter/sink/pulsar/sink.go index b7ea2acaf3..eae48b6130 100644 --- a/downstreamadapter/sink/pulsar/sink.go +++ b/downstreamadapter/sink/pulsar/sink.go @@ -344,6 +344,7 @@ func (s *sink) calculateKeyPartitions(ctx context.Context) error { Event: row, Callback: rowCallback, ColumnSelector: selector, + Checksum: event.Checksum, }, } s.rowChan <- mqEvent diff --git a/eventpb/event.pb.go b/eventpb/event.pb.go index 00486bc89d..45ccfc8edc 100644 --- a/eventpb/event.pb.go +++ b/eventpb/event.pb.go @@ -582,6 +582,58 @@ func (m *EventFeed) GetTableInfo() *TableInfo { return nil } +type IntegrityConfig struct { + IntegrityCheckLevel string `protobuf:"bytes,1,opt,name=IntegrityCheckLevel,proto3" json:"IntegrityCheckLevel,omitempty"` + CorruptionHandleLevel string `protobuf:"bytes,2,opt,name=CorruptionHandleLevel,proto3" json:"CorruptionHandleLevel,omitempty"` +} + +func (m *IntegrityConfig) Reset() { *m = IntegrityConfig{} } +func (m *IntegrityConfig) String() string { return proto.CompactTextString(m) } +func (*IntegrityConfig) ProtoMessage() {} +func (*IntegrityConfig) Descriptor() ([]byte, []int) { + return fileDescriptor_d7fb2554dfcf7f7d, []int{8} +} +func (m *IntegrityConfig) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *IntegrityConfig) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_IntegrityConfig.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *IntegrityConfig) XXX_Merge(src proto.Message) { + xxx_messageInfo_IntegrityConfig.Merge(m, src) +} +func (m *IntegrityConfig) XXX_Size() int { + return m.Size() +} +func (m *IntegrityConfig) XXX_DiscardUnknown() { + xxx_messageInfo_IntegrityConfig.DiscardUnknown(m) +} + +var xxx_messageInfo_IntegrityConfig proto.InternalMessageInfo + +func (m *IntegrityConfig) GetIntegrityCheckLevel() string { + if m != nil { + return m.IntegrityCheckLevel + } + return "" +} + +func (m *IntegrityConfig) GetCorruptionHandleLevel() string { + if m != nil { + return m.CorruptionHandleLevel + } + return "" +} + type RegisterDispatcherRequest struct { ChangefeedId *heartbeatpb.ChangefeedID `protobuf:"bytes,1,opt,name=changefeed_id,json=changefeedId,proto3" json:"changefeed_id,omitempty"` DispatcherId *heartbeatpb.DispatcherID `protobuf:"bytes,2,opt,name=dispatcher_id,json=dispatcherId,proto3" json:"dispatcher_id,omitempty"` @@ -595,13 +647,15 @@ type RegisterDispatcherRequest struct { SyncPointInterval uint64 `protobuf:"varint,10,opt,name=sync_point_interval,json=syncPointInterval,proto3" json:"sync_point_interval,omitempty"` OnlyReuse bool `protobuf:"varint,11,opt,name=only_reuse,json=onlyReuse,proto3" json:"only_reuse,omitempty"` BdrMode bool `protobuf:"varint,12,opt,name=bdr_mode,json=bdrMode,proto3" json:"bdr_mode,omitempty"` + Integrity *IntegrityConfig `protobuf:"bytes,13,opt,name=integrity,proto3" json:"integrity,omitempty"` + Timezone string `protobuf:"bytes,14,opt,name=timezone,proto3" json:"timezone,omitempty"` } func (m *RegisterDispatcherRequest) Reset() { *m = RegisterDispatcherRequest{} } func (m *RegisterDispatcherRequest) String() string { return proto.CompactTextString(m) } func (*RegisterDispatcherRequest) ProtoMessage() {} func (*RegisterDispatcherRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_d7fb2554dfcf7f7d, []int{8} + return fileDescriptor_d7fb2554dfcf7f7d, []int{9} } func (m *RegisterDispatcherRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -714,6 +768,20 @@ func (m *RegisterDispatcherRequest) GetBdrMode() bool { return false } +func (m *RegisterDispatcherRequest) GetIntegrity() *IntegrityConfig { + if m != nil { + return m.Integrity + } + return nil +} + +func (m *RegisterDispatcherRequest) GetTimezone() string { + if m != nil { + return m.Timezone + } + return "" +} + func init() { proto.RegisterEnum("eventpb.OpType", OpType_name, OpType_value) proto.RegisterEnum("eventpb.ActionType", ActionType_name, ActionType_value) @@ -725,79 +793,85 @@ func init() { proto.RegisterType((*TxnEvent)(nil), "eventpb.TxnEvent") proto.RegisterType((*TableInfo)(nil), "eventpb.TableInfo") proto.RegisterType((*EventFeed)(nil), "eventpb.EventFeed") + proto.RegisterType((*IntegrityConfig)(nil), "eventpb.IntegrityConfig") proto.RegisterType((*RegisterDispatcherRequest)(nil), "eventpb.RegisterDispatcherRequest") } func init() { proto.RegisterFile("eventpb/event.proto", fileDescriptor_d7fb2554dfcf7f7d) } var fileDescriptor_d7fb2554dfcf7f7d = []byte{ - // 1042 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xdf, 0x6e, 0xe3, 0xc4, - 0x17, 0xae, 0x93, 0x34, 0x7f, 0x4e, 0xd2, 0xad, 0x3b, 0xdd, 0xee, 0xcf, 0x6d, 0x77, 0xf3, 0xcb, - 0x46, 0x68, 0x15, 0x2a, 0x91, 0x42, 0x01, 0x21, 0xad, 0x50, 0xa5, 0xd2, 0xba, 0x8b, 0x2f, 0xfa, - 0x47, 0x13, 0x77, 0x25, 0xb8, 0xb1, 0x1c, 0xfb, 0x24, 0x35, 0xb8, 0x63, 0xd7, 0x33, 0xc9, 0x36, - 0x6f, 0xc1, 0x15, 0x57, 0x3c, 0x07, 0xcf, 0xc0, 0xe5, 0x5e, 0x72, 0x07, 0x6a, 0x25, 0x78, 0x0d, - 0xe4, 0x19, 0xc7, 0x49, 0x1a, 0x40, 0xe2, 0x2a, 0x33, 0xe7, 0xfb, 0xce, 0xcc, 0x77, 0xbe, 0x73, - 0xc6, 0x2d, 0x6c, 0xe2, 0x18, 0x99, 0x88, 0xfb, 0xfb, 0xf2, 0xb7, 0x1b, 0x27, 0x91, 0x88, 0x48, - 0x25, 0x0b, 0xee, 0xec, 0x5e, 0xa3, 0x9b, 0x88, 0x3e, 0xba, 0x29, 0x23, 0x5f, 0x2b, 0x56, 0xfb, - 0xb7, 0x02, 0xac, 0x9b, 0x29, 0xf1, 0x34, 0x08, 0x05, 0x26, 0x74, 0x14, 0x22, 0x31, 0xa0, 0x72, - 0xe3, 0x0a, 0xef, 0x1a, 0x13, 0x43, 0x6b, 0x15, 0x3b, 0x35, 0x3a, 0xdd, 0x92, 0x97, 0xd0, 0x08, - 0x86, 0x2c, 0x4a, 0xd0, 0x91, 0x87, 0x1b, 0x05, 0x09, 0xd7, 0x55, 0x4c, 0x1e, 0x43, 0x5e, 0x00, - 0x64, 0x14, 0x7e, 0x1b, 0x1a, 0x45, 0x49, 0xa8, 0xa9, 0x48, 0xef, 0x36, 0x24, 0x5f, 0x80, 0x91, - 0xc1, 0x01, 0xe3, 0x98, 0x08, 0x67, 0xec, 0x86, 0x23, 0x74, 0xf0, 0x2e, 0x4e, 0x8c, 0x52, 0x4b, - 0xeb, 0xd4, 0xe8, 0x96, 0xc2, 0x2d, 0x09, 0xbf, 0x4d, 0x51, 0xf3, 0x2e, 0x4e, 0xc8, 0x21, 0x3c, - 0xcf, 0x12, 0x47, 0xb1, 0xef, 0x0a, 0x74, 0x18, 0xbe, 0x9b, 0x4f, 0x5e, 0x95, 0xc9, 0xd9, 0xe1, - 0x57, 0x92, 0x72, 0x8e, 0xef, 0xfe, 0x25, 0x3f, 0x0a, 0xfd, 0xf9, 0xfc, 0xf2, 0x72, 0xfe, 0x45, - 0xe8, 0xcf, 0xf2, 0x67, 0xc2, 0x7d, 0x0c, 0x51, 0xe0, 0x7c, 0x6e, 0x65, 0x5e, 0xf8, 0x89, 0x84, - 0xf3, 0xc4, 0xf6, 0x8f, 0x1a, 0x6c, 0x58, 0x8c, 0x61, 0xa2, 0x1c, 0x3e, 0x8e, 0xd8, 0x20, 0x18, - 0x92, 0xa7, 0xb0, 0x9a, 0x8c, 0x42, 0xe4, 0x99, 0xc3, 0x6a, 0x43, 0x3e, 0x82, 0xcd, 0xec, 0x12, - 0x71, 0xc7, 0x1c, 0x2e, 0xdc, 0x44, 0x38, 0x82, 0x4b, 0x9b, 0x4b, 0x54, 0x57, 0x90, 0x7d, 0xc7, - 0x7a, 0x29, 0x60, 0x73, 0xf2, 0x25, 0x34, 0xe6, 0x7a, 0xc7, 0xa5, 0xdb, 0xf5, 0x03, 0xa3, 0x9b, - 0x75, 0xbe, 0xfb, 0xa8, 0xb1, 0x74, 0x81, 0xdd, 0xfe, 0x49, 0x83, 0xc6, 0x82, 0xa6, 0x0f, 0x60, - 0xcd, 0x73, 0x39, 0xf6, 0x90, 0xf1, 0x40, 0x04, 0x63, 0x34, 0xb4, 0x96, 0xd6, 0xa9, 0xd2, 0xc5, - 0x20, 0x79, 0x05, 0x4f, 0x06, 0x51, 0xe2, 0x21, 0xc5, 0x38, 0x0c, 0x3c, 0x57, 0xa0, 0x51, 0x90, - 0xb4, 0x47, 0x51, 0x72, 0x08, 0x8d, 0xc1, 0xdc, 0xe9, 0x46, 0xb1, 0xa5, 0x75, 0xea, 0x07, 0x3b, - 0xb9, 0xb8, 0x25, 0x4f, 0xe8, 0x02, 0xbf, 0xdd, 0x00, 0xa0, 0xc8, 0xa3, 0x70, 0x8c, 0xbe, 0xcd, - 0xdb, 0x23, 0x58, 0x55, 0xf3, 0xa5, 0x43, 0xf1, 0x7b, 0x9c, 0x48, 0x69, 0x0d, 0x9a, 0x2e, 0x53, - 0x2b, 0x65, 0x2f, 0xa4, 0x8e, 0x06, 0x55, 0x1b, 0xb2, 0x03, 0xd5, 0x69, 0xff, 0xe4, 0xd5, 0x0d, - 0x9a, 0xef, 0x49, 0x07, 0x2a, 0x51, 0xec, 0x88, 0x49, 0x8c, 0x72, 0xe6, 0x9e, 0x1c, 0xac, 0xe7, - 0xaa, 0x2e, 0x62, 0x7b, 0x12, 0x23, 0x2d, 0x47, 0xf2, 0xb7, 0xfd, 0x1d, 0x54, 0xed, 0x3b, 0xa6, - 0x6e, 0x7e, 0x05, 0x65, 0xc9, 0x52, 0x3d, 0xab, 0x1f, 0x3c, 0x59, 0xf4, 0x99, 0x66, 0x28, 0xd9, - 0x85, 0x9a, 0x17, 0xdd, 0xdc, 0x04, 0x59, 0xeb, 0xb4, 0x4e, 0x89, 0x56, 0x55, 0xc0, 0xe6, 0x64, - 0x1b, 0xaa, 0x79, 0x5b, 0x8b, 0x12, 0xab, 0x70, 0xd5, 0xcd, 0x76, 0x1d, 0x6a, 0xb6, 0xdb, 0x0f, - 0xd1, 0x62, 0x83, 0xa8, 0xfd, 0xa7, 0x06, 0x35, 0xd5, 0x2d, 0x44, 0x9f, 0x7c, 0x0c, 0x90, 0x0e, - 0xc4, 0xc2, 0xf5, 0x1b, 0xf9, 0xf5, 0x53, 0x85, 0xb4, 0x26, 0xb2, 0x15, 0x27, 0xff, 0x87, 0x7a, - 0x92, 0xb9, 0x37, 0x93, 0x01, 0x49, 0x6e, 0x28, 0x39, 0x84, 0x35, 0x3f, 0xe0, 0xb1, 0x7a, 0xd8, - 0x4e, 0xe0, 0x67, 0xfd, 0xd9, 0xee, 0xce, 0x7d, 0x2d, 0xba, 0x27, 0x39, 0xc3, 0x3a, 0xa1, 0x8d, - 0x19, 0xdf, 0xf2, 0xe5, 0x00, 0xbb, 0x22, 0x88, 0xa4, 0x83, 0x05, 0xaa, 0x36, 0xe4, 0x13, 0x00, - 0x91, 0xd6, 0xe0, 0x04, 0x6c, 0x10, 0xc9, 0x37, 0x59, 0x3f, 0x20, 0x33, 0xa1, 0xd3, 0xf2, 0x68, - 0x4d, 0xe4, 0x95, 0xfe, 0x5c, 0x82, 0x6d, 0x8a, 0xc3, 0x80, 0x0b, 0x4c, 0x66, 0xf7, 0x51, 0xbc, - 0x1d, 0x21, 0x17, 0xa9, 0x4c, 0xef, 0xda, 0x65, 0x43, 0x1c, 0x20, 0xfa, 0xa9, 0x4c, 0xed, 0x6f, - 0x64, 0x1e, 0xe7, 0x8c, 0x54, 0xe6, 0x8c, 0x6f, 0xf9, 0xcb, 0x65, 0x16, 0xfe, 0x5b, 0x99, 0x9f, - 0x4f, 0x0b, 0xe2, 0xb1, 0xcb, 0x32, 0x8f, 0x9e, 0x2d, 0x24, 0xcb, 0xa2, 0x7a, 0xb1, 0xcb, 0xb2, - 0xa2, 0xd2, 0xe5, 0x42, 0x9b, 0x4b, 0x0b, 0x6d, 0x4e, 0xc7, 0x83, 0x63, 0x32, 0x56, 0x6a, 0xd4, - 0x57, 0xab, 0xaa, 0x02, 0x96, 0x4f, 0x3e, 0x83, 0xba, 0xeb, 0x89, 0x20, 0x62, 0x6a, 0x3a, 0xcb, - 0x72, 0x3a, 0x37, 0x73, 0x03, 0x8f, 0x24, 0x26, 0x27, 0x14, 0xdc, 0x7c, 0x4d, 0x5e, 0xc3, 0x9a, - 0x7a, 0x3a, 0x8e, 0xa7, 0xde, 0x5a, 0x45, 0xea, 0xdc, 0xca, 0xf3, 0xfe, 0xf9, 0x99, 0x91, 0x3d, - 0xd8, 0x40, 0xa6, 0x2a, 0x9c, 0x30, 0xcf, 0x89, 0xa3, 0x80, 0x09, 0xa3, 0x2a, 0x5f, 0xf4, 0xba, - 0x02, 0x7a, 0x13, 0xe6, 0x5d, 0xa6, 0x61, 0xd2, 0x86, 0xb5, 0x19, 0x29, 0x2d, 0xad, 0x26, 0x4b, - 0xab, 0xf3, 0x29, 0xc3, 0xe6, 0xa4, 0x0b, 0x9b, 0x73, 0x9c, 0x80, 0x09, 0x4c, 0xc6, 0x6e, 0x68, - 0x80, 0x64, 0x6e, 0xe4, 0x4c, 0x2b, 0x03, 0xd2, 0xbf, 0x17, 0x11, 0x0b, 0x27, 0x4e, 0x82, 0x23, - 0x8e, 0x46, 0x5d, 0x5e, 0x5c, 0x4b, 0x23, 0x34, 0x0d, 0xa4, 0x46, 0xf6, 0xfd, 0xc4, 0xb9, 0x89, - 0x7c, 0x34, 0x1a, 0x12, 0xac, 0xf4, 0xfd, 0xe4, 0x2c, 0xf2, 0x71, 0xef, 0x43, 0x28, 0xab, 0xd7, - 0x4a, 0xd6, 0xa0, 0xa6, 0x56, 0x97, 0x23, 0xa1, 0xaf, 0x10, 0x1d, 0x1a, 0x6a, 0xab, 0x3e, 0xc5, - 0xba, 0xb6, 0xf7, 0x87, 0x06, 0x30, 0xf3, 0x8e, 0xec, 0xc2, 0xff, 0x8e, 0x8e, 0x6d, 0xeb, 0xe2, - 0xdc, 0xb1, 0xbf, 0xb9, 0x34, 0x9d, 0xab, 0xf3, 0xde, 0xa5, 0x79, 0x6c, 0x9d, 0x5a, 0xe6, 0x89, - 0xbe, 0x42, 0x0c, 0x78, 0x3a, 0x0f, 0x52, 0xf3, 0x8d, 0xd5, 0xb3, 0x4d, 0xaa, 0x6b, 0xe4, 0x19, - 0x90, 0x45, 0xe4, 0xec, 0xe2, 0xad, 0xa9, 0x17, 0xc8, 0x16, 0x6c, 0xcc, 0xc7, 0x2f, 0x8f, 0xae, - 0x7a, 0xa6, 0x5e, 0x5c, 0xa6, 0xf7, 0xae, 0xce, 0x4c, 0xbd, 0xf4, 0x98, 0x4e, 0xcd, 0x9e, 0x69, - 0xeb, 0xab, 0xa4, 0x05, 0xcf, 0x97, 0x4e, 0x71, 0x8e, 0xbf, 0x3e, 0x3a, 0x7f, 0x63, 0x9e, 0x9a, - 0xe6, 0x89, 0x5e, 0x26, 0x2f, 0xe1, 0xc5, 0xf2, 0x81, 0xf3, 0x94, 0xca, 0x57, 0xaf, 0x7f, 0xb9, - 0x6f, 0x6a, 0xef, 0xef, 0x9b, 0xda, 0xef, 0xf7, 0x4d, 0xed, 0x87, 0x87, 0xe6, 0xca, 0xfb, 0x87, - 0xe6, 0xca, 0xaf, 0x0f, 0xcd, 0x95, 0x6f, 0x5b, 0xc3, 0x40, 0x5c, 0x8f, 0xfa, 0x5d, 0x2f, 0xba, - 0xd9, 0x8f, 0x03, 0x36, 0xf4, 0xdc, 0x78, 0x5f, 0x04, 0x9e, 0xef, 0xed, 0x67, 0x43, 0xd2, 0x2f, - 0xcb, 0xff, 0x08, 0x3e, 0xfd, 0x2b, 0x00, 0x00, 0xff, 0xff, 0xea, 0xda, 0xde, 0xb3, 0x4e, 0x08, - 0x00, 0x00, + // 1127 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x56, 0xcf, 0x6e, 0xdb, 0xc6, + 0x13, 0x36, 0x25, 0x5b, 0x12, 0x47, 0x92, 0x4d, 0xaf, 0xe3, 0xfc, 0x98, 0x7f, 0xfa, 0x29, 0x42, + 0x11, 0xa8, 0x01, 0x2a, 0xa7, 0x6e, 0xda, 0x02, 0x41, 0x11, 0xc0, 0x95, 0x99, 0x84, 0x40, 0x63, + 0x1b, 0x2b, 0x3a, 0x40, 0x7b, 0x21, 0x28, 0x72, 0x24, 0xb3, 0xa1, 0x96, 0x0c, 0xb9, 0x52, 0xac, + 0x3e, 0x42, 0x4f, 0x3d, 0xf5, 0xd4, 0x07, 0xea, 0x31, 0xc7, 0xde, 0x5a, 0x24, 0x40, 0xfb, 0x1a, + 0x05, 0x77, 0x29, 0x8a, 0xb2, 0xdc, 0x02, 0x3d, 0x69, 0x77, 0xbe, 0x6f, 0x76, 0x67, 0xbe, 0x99, + 0x59, 0x0a, 0xf6, 0x70, 0x86, 0x8c, 0x47, 0xc3, 0x03, 0xf1, 0xdb, 0x8b, 0xe2, 0x90, 0x87, 0xa4, + 0x9a, 0x19, 0x6f, 0xdf, 0xb9, 0x40, 0x27, 0xe6, 0x43, 0x74, 0x52, 0x46, 0xbe, 0x96, 0xac, 0xce, + 0xef, 0x25, 0xd8, 0x31, 0x52, 0xe2, 0x33, 0x3f, 0xe0, 0x18, 0xd3, 0x69, 0x80, 0x44, 0x87, 0xea, + 0xc4, 0xe1, 0xee, 0x05, 0xc6, 0xba, 0xd2, 0x2e, 0x77, 0x55, 0xba, 0xd8, 0x92, 0xfb, 0xd0, 0xf0, + 0xc7, 0x2c, 0x8c, 0xd1, 0x16, 0x87, 0xeb, 0x25, 0x01, 0xd7, 0xa5, 0x4d, 0x1c, 0x43, 0xee, 0x01, + 0x64, 0x94, 0xe4, 0x4d, 0xa0, 0x97, 0x05, 0x41, 0x95, 0x96, 0xc1, 0x9b, 0x80, 0x7c, 0x09, 0x7a, + 0x06, 0xfb, 0x2c, 0xc1, 0x98, 0xdb, 0x33, 0x27, 0x98, 0xa2, 0x8d, 0x97, 0x51, 0xac, 0x6f, 0xb6, + 0x95, 0xae, 0x4a, 0xf7, 0x25, 0x6e, 0x0a, 0xf8, 0x55, 0x8a, 0x1a, 0x97, 0x51, 0x4c, 0x9e, 0xc2, + 0xdd, 0xcc, 0x71, 0x1a, 0x79, 0x0e, 0x47, 0x9b, 0xe1, 0xdb, 0xa2, 0xf3, 0x96, 0x70, 0xce, 0x0e, + 0x3f, 0x17, 0x94, 0x13, 0x7c, 0xfb, 0x2f, 0xfe, 0x61, 0xe0, 0x15, 0xfd, 0x2b, 0xeb, 0xfe, 0xa7, + 0x81, 0xb7, 0xf4, 0x5f, 0x06, 0xee, 0x61, 0x80, 0x1c, 0x8b, 0xbe, 0xd5, 0x62, 0xe0, 0xc7, 0x02, + 0xce, 0x1d, 0x3b, 0x3f, 0x2b, 0xb0, 0x6b, 0x32, 0x86, 0xb1, 0x54, 0xb8, 0x1f, 0xb2, 0x91, 0x3f, + 0x26, 0x37, 0x60, 0x2b, 0x9e, 0x06, 0x98, 0x64, 0x0a, 0xcb, 0x0d, 0xf9, 0x04, 0xf6, 0xb2, 0x4b, + 0xf8, 0x25, 0xb3, 0x13, 0xee, 0xc4, 0xdc, 0xe6, 0x89, 0x90, 0x79, 0x93, 0x6a, 0x12, 0xb2, 0x2e, + 0xd9, 0x20, 0x05, 0xac, 0x84, 0x7c, 0x05, 0x8d, 0x42, 0xed, 0x12, 0xa1, 0x76, 0xfd, 0x50, 0xef, + 0x65, 0x95, 0xef, 0x5d, 0x29, 0x2c, 0x5d, 0x61, 0x77, 0x7e, 0x51, 0xa0, 0xb1, 0x12, 0xd3, 0x47, + 0xd0, 0x74, 0x9d, 0x04, 0x07, 0xc8, 0x12, 0x9f, 0xfb, 0x33, 0xd4, 0x95, 0xb6, 0xd2, 0xad, 0xd1, + 0x55, 0x23, 0x79, 0x00, 0xdb, 0xa3, 0x30, 0x76, 0x91, 0x62, 0x14, 0xf8, 0xae, 0xc3, 0x51, 0x2f, + 0x09, 0xda, 0x15, 0x2b, 0x79, 0x0a, 0x8d, 0x51, 0xe1, 0x74, 0xbd, 0xdc, 0x56, 0xba, 0xf5, 0xc3, + 0xdb, 0x79, 0x70, 0x6b, 0x9a, 0xd0, 0x15, 0x7e, 0xa7, 0x01, 0x40, 0x31, 0x09, 0x83, 0x19, 0x7a, + 0x56, 0xd2, 0x99, 0xc2, 0x96, 0xec, 0x2f, 0x0d, 0xca, 0xaf, 0x71, 0x2e, 0x42, 0x6b, 0xd0, 0x74, + 0x99, 0x4a, 0x29, 0x6a, 0x21, 0xe2, 0x68, 0x50, 0xb9, 0x21, 0xb7, 0xa1, 0xb6, 0xa8, 0x9f, 0xb8, + 0xba, 0x41, 0xf3, 0x3d, 0xe9, 0x42, 0x35, 0x8c, 0x6c, 0x3e, 0x8f, 0x50, 0xf4, 0xdc, 0xf6, 0xe1, + 0x4e, 0x1e, 0xd5, 0x69, 0x64, 0xcd, 0x23, 0xa4, 0x95, 0x50, 0xfc, 0x76, 0xbe, 0x87, 0x9a, 0x75, + 0xc9, 0xe4, 0xcd, 0x0f, 0xa0, 0x22, 0x58, 0xb2, 0x66, 0xf5, 0xc3, 0xed, 0x55, 0x9d, 0x69, 0x86, + 0x92, 0x3b, 0xa0, 0xba, 0xe1, 0x64, 0xe2, 0x67, 0xa5, 0x53, 0xba, 0x9b, 0xb4, 0x26, 0x0d, 0x56, + 0x42, 0x6e, 0x41, 0x2d, 0x2f, 0x6b, 0x59, 0x60, 0xd5, 0x44, 0x56, 0xb3, 0x53, 0x07, 0xd5, 0x72, + 0x86, 0x01, 0x9a, 0x6c, 0x14, 0x76, 0xfe, 0x52, 0x40, 0x95, 0xd5, 0x42, 0xf4, 0xc8, 0x23, 0x80, + 0xb4, 0x21, 0x56, 0xae, 0xdf, 0xcd, 0xaf, 0x5f, 0x44, 0x48, 0x55, 0x9e, 0xad, 0x12, 0xf2, 0x7f, + 0xa8, 0xc7, 0x99, 0x7a, 0xcb, 0x30, 0x20, 0xce, 0x05, 0x25, 0x4f, 0xa1, 0xe9, 0xf9, 0x49, 0x24, + 0x07, 0xdb, 0xf6, 0xbd, 0xac, 0x3e, 0xb7, 0x7a, 0x85, 0xd7, 0xa2, 0x77, 0x9c, 0x33, 0xcc, 0x63, + 0xda, 0x58, 0xf2, 0x4d, 0x4f, 0x34, 0xb0, 0xc3, 0xfd, 0x50, 0x28, 0x58, 0xa2, 0x72, 0x43, 0x3e, + 0x05, 0xe0, 0x69, 0x0e, 0xb6, 0xcf, 0x46, 0xa1, 0x98, 0xc9, 0xfa, 0x21, 0x59, 0x06, 0xba, 0x48, + 0x8f, 0xaa, 0x3c, 0xcf, 0x74, 0x0e, 0x3b, 0x26, 0xe3, 0x38, 0x8e, 0x7d, 0x3e, 0xcf, 0x1a, 0xf1, + 0x11, 0xec, 0x2d, 0x4d, 0x17, 0xe8, 0xbe, 0xfe, 0x06, 0x67, 0x18, 0x88, 0x9a, 0xab, 0xf4, 0x3a, + 0x88, 0x3c, 0x86, 0xfd, 0x7e, 0x18, 0xc7, 0xd3, 0x88, 0xfb, 0x21, 0x7b, 0xe1, 0x30, 0x2f, 0x40, + 0xe9, 0x53, 0x92, 0xa3, 0x79, 0x2d, 0xd8, 0xf9, 0x71, 0x0b, 0x6e, 0x51, 0x1c, 0xfb, 0x09, 0xc7, + 0x78, 0x99, 0x2a, 0xc5, 0x37, 0x53, 0x4c, 0x78, 0xaa, 0x90, 0x7b, 0xe1, 0xb0, 0x31, 0x8e, 0x10, + 0xbd, 0x54, 0x21, 0xe5, 0x1a, 0x85, 0xfa, 0x39, 0x23, 0x55, 0x68, 0xc9, 0x37, 0xbd, 0x75, 0x85, + 0x4b, 0xff, 0x4d, 0xe1, 0xcf, 0x17, 0x5a, 0x26, 0x91, 0xc3, 0xb2, 0xf2, 0xdc, 0x5c, 0x71, 0x16, + 0x7a, 0x0e, 0x22, 0x87, 0x65, 0x7a, 0xa6, 0xcb, 0x95, 0x0e, 0xdb, 0x5c, 0xe9, 0xb0, 0xb4, 0x33, + 0x13, 0x8c, 0x67, 0x32, 0x1a, 0xf9, 0x60, 0xd6, 0xa4, 0xc1, 0xf4, 0xc8, 0x63, 0xa8, 0x3b, 0x6e, + 0xaa, 0x90, 0x1c, 0x8c, 0x8a, 0x18, 0x8c, 0xbd, 0xbc, 0x76, 0x47, 0x02, 0x13, 0xc3, 0x01, 0x4e, + 0xbe, 0x26, 0x4f, 0xa0, 0x29, 0xa7, 0xd6, 0x76, 0xe5, 0x98, 0x57, 0x45, 0x9c, 0xfb, 0xb9, 0xdf, + 0x3f, 0x4f, 0x38, 0x79, 0x08, 0xbb, 0xc8, 0x64, 0x86, 0x73, 0xe6, 0xda, 0x51, 0xe8, 0x33, 0xae, + 0xd7, 0xc4, 0x63, 0xb2, 0x23, 0x81, 0xc1, 0x9c, 0xb9, 0x67, 0xa9, 0x99, 0x74, 0xa0, 0xb9, 0x24, + 0xa5, 0xa9, 0xa9, 0x22, 0xb5, 0x7a, 0xb2, 0x60, 0x58, 0x09, 0xe9, 0xc1, 0x5e, 0x81, 0xe3, 0x33, + 0x8e, 0xf1, 0xcc, 0x09, 0x74, 0x10, 0xcc, 0xdd, 0x9c, 0x69, 0x66, 0x40, 0xfa, 0xa9, 0x0a, 0x59, + 0x30, 0xb7, 0x63, 0x9c, 0x26, 0xa8, 0xd7, 0xc5, 0xc5, 0x6a, 0x6a, 0xa1, 0xa9, 0x21, 0x15, 0x72, + 0xe8, 0xc5, 0xf6, 0x24, 0xf4, 0x50, 0x6f, 0x08, 0xb0, 0x3a, 0xf4, 0xe2, 0x97, 0xa1, 0x87, 0xe4, + 0x0b, 0x50, 0xfd, 0x45, 0x17, 0xea, 0x4d, 0x91, 0xb1, 0x5e, 0x78, 0xd8, 0x56, 0xba, 0x99, 0x2e, + 0xa9, 0xe9, 0xa3, 0xc4, 0xfd, 0x09, 0xfe, 0x10, 0x32, 0xd4, 0xb7, 0xa5, 0xfe, 0x8b, 0xfd, 0xc3, + 0x8f, 0xa1, 0x22, 0x1f, 0x1f, 0xd2, 0x04, 0x55, 0xae, 0xce, 0xa6, 0x5c, 0xdb, 0x20, 0x1a, 0x34, + 0xe4, 0x56, 0x7e, 0x59, 0x34, 0xe5, 0xe1, 0x9f, 0x0a, 0xc0, 0xb2, 0x1e, 0xe4, 0x0e, 0xfc, 0xef, + 0xa8, 0x6f, 0x99, 0xa7, 0x27, 0xb6, 0xf5, 0xed, 0x99, 0x61, 0x9f, 0x9f, 0x0c, 0xce, 0x8c, 0xbe, + 0xf9, 0xcc, 0x34, 0x8e, 0xb5, 0x0d, 0xa2, 0xc3, 0x8d, 0x22, 0x48, 0x8d, 0xe7, 0xe6, 0xc0, 0x32, + 0xa8, 0xa6, 0x90, 0x9b, 0x40, 0x56, 0x91, 0x97, 0xa7, 0xaf, 0x0c, 0xad, 0x44, 0xf6, 0x61, 0xb7, + 0x68, 0x3f, 0x3b, 0x3a, 0x1f, 0x18, 0x5a, 0x79, 0x9d, 0x3e, 0x38, 0x7f, 0x69, 0x68, 0x9b, 0x57, + 0xe9, 0xd4, 0x18, 0x18, 0x96, 0xb6, 0x45, 0xda, 0x70, 0x77, 0xed, 0x14, 0xbb, 0xff, 0xe2, 0xe8, + 0xe4, 0xb9, 0xf1, 0xcc, 0x30, 0x8e, 0xb5, 0x0a, 0xb9, 0x0f, 0xf7, 0xd6, 0x0f, 0x2c, 0x52, 0xaa, + 0x5f, 0x3f, 0xf9, 0xf5, 0x7d, 0x4b, 0x79, 0xf7, 0xbe, 0xa5, 0xfc, 0xf1, 0xbe, 0xa5, 0xfc, 0xf4, + 0xa1, 0xb5, 0xf1, 0xee, 0x43, 0x6b, 0xe3, 0xb7, 0x0f, 0xad, 0x8d, 0xef, 0xda, 0x63, 0x9f, 0x5f, + 0x4c, 0x87, 0x3d, 0x37, 0x9c, 0x1c, 0x44, 0x3e, 0x1b, 0xbb, 0x4e, 0x74, 0xc0, 0x7d, 0xd7, 0x73, + 0x0f, 0xb2, 0x32, 0x0c, 0x2b, 0xe2, 0x0f, 0xce, 0x67, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0x81, + 0xc5, 0x74, 0xfe, 0x1d, 0x09, 0x00, 0x00, } func (m *EventFilterRule) Marshal() (dAtA []byte, err error) { @@ -1211,6 +1285,43 @@ func (m *EventFeed) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *IntegrityConfig) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *IntegrityConfig) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *IntegrityConfig) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.CorruptionHandleLevel) > 0 { + i -= len(m.CorruptionHandleLevel) + copy(dAtA[i:], m.CorruptionHandleLevel) + i = encodeVarintEvent(dAtA, i, uint64(len(m.CorruptionHandleLevel))) + i-- + dAtA[i] = 0x12 + } + if len(m.IntegrityCheckLevel) > 0 { + i -= len(m.IntegrityCheckLevel) + copy(dAtA[i:], m.IntegrityCheckLevel) + i = encodeVarintEvent(dAtA, i, uint64(len(m.IntegrityCheckLevel))) + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} + func (m *RegisterDispatcherRequest) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -1231,6 +1342,25 @@ func (m *RegisterDispatcherRequest) MarshalToSizedBuffer(dAtA []byte) (int, erro _ = i var l int _ = l + if len(m.Timezone) > 0 { + i -= len(m.Timezone) + copy(dAtA[i:], m.Timezone) + i = encodeVarintEvent(dAtA, i, uint64(len(m.Timezone))) + i-- + dAtA[i] = 0x72 + } + if m.Integrity != nil { + { + size, err := m.Integrity.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintEvent(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x6a + } if m.BdrMode { i-- if m.BdrMode { @@ -1532,6 +1662,23 @@ func (m *EventFeed) Size() (n int) { return n } +func (m *IntegrityConfig) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.IntegrityCheckLevel) + if l > 0 { + n += 1 + l + sovEvent(uint64(l)) + } + l = len(m.CorruptionHandleLevel) + if l > 0 { + n += 1 + l + sovEvent(uint64(l)) + } + return n +} + func (m *RegisterDispatcherRequest) Size() (n int) { if m == nil { return 0 @@ -1579,6 +1726,14 @@ func (m *RegisterDispatcherRequest) Size() (n int) { if m.BdrMode { n += 2 } + if m.Integrity != nil { + l = m.Integrity.Size() + n += 1 + l + sovEvent(uint64(l)) + } + l = len(m.Timezone) + if l > 0 { + n += 1 + l + sovEvent(uint64(l)) + } return n } @@ -2759,6 +2914,120 @@ func (m *EventFeed) Unmarshal(dAtA []byte) error { } return nil } +func (m *IntegrityConfig) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvent + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: IntegrityConfig: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: IntegrityConfig: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field IntegrityCheckLevel", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvent + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthEvent + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEvent + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.IntegrityCheckLevel = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CorruptionHandleLevel", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvent + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthEvent + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEvent + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.CorruptionHandleLevel = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipEvent(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthEvent + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *RegisterDispatcherRequest) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -3100,6 +3369,74 @@ func (m *RegisterDispatcherRequest) Unmarshal(dAtA []byte) error { } } m.BdrMode = bool(v != 0) + case 13: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Integrity", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvent + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthEvent + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthEvent + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Integrity == nil { + m.Integrity = &IntegrityConfig{} + } + if err := m.Integrity.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 14: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Timezone", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowEvent + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthEvent + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthEvent + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Timezone = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipEvent(dAtA[iNdEx:]) diff --git a/eventpb/event.proto b/eventpb/event.proto index 31b7ff4483..e678876988 100644 --- a/eventpb/event.proto +++ b/eventpb/event.proto @@ -75,6 +75,11 @@ enum ActionType { ACTION_TYPE_RESUME_CHANGEFEED = 7; } +message IntegrityConfig { + string IntegrityCheckLevel = 1; + string CorruptionHandleLevel = 2; +} + message RegisterDispatcherRequest { heartbeatpb.ChangefeedID changefeed_id = 1; heartbeatpb.DispatcherID dispatcher_id = 2; @@ -88,4 +93,6 @@ message RegisterDispatcherRequest { uint64 sync_point_interval = 10; bool only_reuse = 11; bool bdr_mode = 12; + IntegrityConfig integrity = 13; + string timezone = 14; } diff --git a/logservice/eventstore/event_store.go b/logservice/eventstore/event_store.go index cbfb1097b5..d97554b2ba 100644 --- a/logservice/eventstore/event_store.go +++ b/logservice/eventstore/event_store.go @@ -31,7 +31,6 @@ import ( "github.com/pingcap/ticdc/logservice/logservicepb" "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" - "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/messaging" "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" @@ -617,7 +616,6 @@ func (e *eventStore) GetIterator(dispatcherID common.DispatcherID, dataRange com innerIter: iter, prevStartTs: 0, prevCommitTs: 0, - iterMounter: event.NewMounter(time.Local), // FIXME startTs: dataRange.StartTs, endTs: dataRange.EndTs, rowCount: 0, @@ -714,7 +712,6 @@ type eventStoreIter struct { innerIter *pebble.Iterator prevStartTs uint64 prevCommitTs uint64 - iterMounter event.Mounter // for debug startTs uint64 diff --git a/pkg/common/event/checksum.go b/pkg/common/event/checksum.go new file mode 100644 index 0000000000..a6260fd11e --- /dev/null +++ b/pkg/common/event/checksum.go @@ -0,0 +1,175 @@ +// Copyright 2025 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package event + +import ( + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta/model" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/rowcodec" + "go.uber.org/zap" +) + +// return error when calculate the checksum. +// return false if the checksum is not matched +func (m *mounter) verifyChecksum( + tableInfo *common.TableInfo, row chunk.Row, key kv.Key, handle kv.Handle, decoder *rowcodec.ChunkDecoder, +) (uint32, bool, error) { + if !m.integrity.Enabled() { + return 0, true, nil + } + columnInfos := tableInfo.GetColumns() + version := decoder.ChecksumVersion() + switch version { + case 0: + // skip old value checksum verification for the checksum v1, since it cannot handle + // Update / Delete event correctly, after Add Column / Drop column DDL, + // since the table schema does not contain complete column information. + return m.verifyColumnChecksum(columnInfos, row, decoder) + case 1, 2: + + expected, matched, err := verifyRawBytesChecksum(columnInfos, row, decoder, key, handle, m.tz) + if err != nil { + log.Error("calculate raw checksum failed", + zap.Int("version", version), zap.Any("tz", m.tz), zap.Any("handle", handle.String()), + zap.Any("key", key), zap.Any("columns", columnInfos), zap.Error(err)) + return 0, false, errors.Trace(err) + } + if !matched { + return expected, matched, err + } + columnChecksum, err := calculateColumnChecksum(columnInfos, row, m.tz) + if err != nil { + log.Error("failed to calculate column-level checksum, after raw checksum verification passed", + zap.Any("columnsInfo", columnInfos), + zap.Any("tz", m.tz), zap.Error(err)) + return 0, false, errors.Trace(err) + } + return columnChecksum, true, nil + default: + } + return 0, false, errors.Errorf("unknown checksum version %d", version) +} + +func (m *mounter) verifyColumnChecksum( + columnInfos []*model.ColumnInfo, row chunk.Row, + decoder *rowcodec.ChunkDecoder, +) (uint32, bool, error) { + // if the checksum cannot be found, which means the upstream TiDB checksum is not enabled, + // so return matched as true to skip check the event. + first, ok := decoder.GetChecksum() + if !ok { + return 0, true, nil + } + + checksum, err := calculateColumnChecksum(columnInfos, row, m.tz) + if err != nil { + log.Error("failed to calculate the checksum", + zap.Uint32("first", first), zap.Any("columnInfos", columnInfos), zap.Error(err)) + return 0, false, err + } + + // the first checksum matched, it hits in the most case. + if checksum == first { + return checksum, true, nil + } + + extra, ok := decoder.GetExtraChecksum() + if ok && checksum == extra { + return checksum, true, nil + } + + if time.Since(m.lastSkipOldValueTime) > time.Minute { + log.Warn("checksum mismatch on the old value, "+ + "this may caused by Add Column / Drop Column executed, skip verification", + zap.Uint32("checksum", checksum), zap.Uint32("first", first), zap.Uint32("extra", extra), + zap.Any("columnInfos", columnInfos), zap.Any("tz", m.tz)) + m.lastSkipOldValueTime = time.Now() + } + return checksum, true, nil +} + +func calculateColumnChecksum( + columnInfos []*model.ColumnInfo, row chunk.Row, tz *time.Location, +) (uint32, error) { + columns := make([]rowcodec.ColData, 0, row.Len()) + for idx, col := range columnInfos { + datum := row.GetDatum(idx, &col.FieldType) + column := rowcodec.ColData{ + ColumnInfo: col, + Datum: &datum, + } + columns = append(columns, column) + } + // FIXME: why need sort? + // sort.Slice(columns, func(i, j int) bool { + // return columns[i].ID < columns[j].ID + // }) + + calculator := rowcodec.RowData{ + Cols: columns, + Data: make([]byte, 0), + } + + checksum, err := calculator.Checksum(tz) + if err != nil { + return 0, errors.Trace(err) + } + return checksum, nil +} + +func verifyRawBytesChecksum( + columnInfos []*model.ColumnInfo, row chunk.Row, decoder *rowcodec.ChunkDecoder, key kv.Key, handle kv.Handle, tz *time.Location, +) (uint32, bool, error) { + expected, ok := decoder.GetChecksum() + if !ok { + return 0, true, nil + } + var ( + columnIDs []int64 + datums []*types.Datum + ) + for idx, col := range columnInfos { + // TiDB does not encode null value into the bytes, so just ignore it. + if row.IsNull(idx) { + continue + } + columnID := col.ID + datum := row.GetDatum(idx, &col.FieldType) + datums = append(datums, &datum) + columnIDs = append(columnIDs, columnID) + } + obtained, err := decoder.CalculateRawChecksum(tz, columnIDs, datums, key, handle, nil) + if err != nil { + return 0, false, errors.Trace(err) + } + if obtained == expected { + return expected, true, nil + } + + log.Error("raw bytes checksum mismatch", + zap.Int("version", decoder.ChecksumVersion()), + zap.Uint32("expected", expected), + zap.Uint32("obtained", obtained), + zap.Any("columns", columnInfos), + zap.Any("tz", tz)) + + return expected, false, nil +} diff --git a/pkg/common/event/chunk.go b/pkg/common/event/chunk.go index d505a49b42..256eebca1e 100644 --- a/pkg/common/event/chunk.go +++ b/pkg/common/event/chunk.go @@ -65,17 +65,17 @@ func newChunkDecoderV2(tableInfo *common.TableInfo, tz *time.Location) *rowcodec } // rawKVToChunkV2 is used to decode the new format of row data. -func (m *mounter) rawKVToChunkV2(value []byte, tableInfo *common.TableInfo, chk *chunk.Chunk, handle kv.Handle) error { +func (m *mounter) rawKVToChunkV2(value []byte, tableInfo *common.TableInfo, chk *chunk.Chunk, handle kv.Handle) (*rowcodec.ChunkDecoder, error) { if len(value) == 0 { - return nil + return nil, nil } decoder := newChunkDecoderV2(tableInfo, m.tz) // cache it for later use err := decoder.DecodeToChunk(value, handle, chk) if err != nil { - return errors.Trace(err) + return nil, errors.Trace(err) } - return nil + return decoder, nil } // rawKVToChunkV1 is used to decode the old format of row data. diff --git a/pkg/common/event/dml_event.go b/pkg/common/event/dml_event.go index 79cbcb613e..0e4b18f0d6 100644 --- a/pkg/common/event/dml_event.go +++ b/pkg/common/event/dml_event.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/tidb/pkg/util/chunk" "go.uber.org/zap" ) @@ -71,6 +72,10 @@ type DMLEvent struct { // offset is the offset of the current row in the transaction. // It is internal field, not exported. So it doesn't need to be marshalled. offset int `json:"-"` + + // Checksum for the event, only not nil if the upstream TiDB enable the row level checksum + // and TiCDC set the integrity check level to the correctness. + Checksum *integrity.Checksum } func NewDMLEvent( @@ -98,7 +103,7 @@ func NewDMLEvent( func (t *DMLEvent) AppendRow(raw *common.RawKVEntry, decode func( rawKv *common.RawKVEntry, - tableInfo *common.TableInfo, chk *chunk.Chunk) (int, error), + tableInfo *common.TableInfo, chk *chunk.Chunk) (int, *integrity.Checksum, error), ) error { rowType := RowTypeInsert if raw.OpType == common.OpTypeDelete { @@ -107,7 +112,7 @@ func (t *DMLEvent) AppendRow(raw *common.RawKVEntry, if len(raw.Value) != 0 && len(raw.OldValue) != 0 { rowType = RowTypeUpdate } - count, err := decode(raw, t.TableInfo, t.Rows) + count, checksum, err := decode(raw, t.TableInfo, t.Rows) if err != nil { return err } @@ -116,6 +121,7 @@ func (t *DMLEvent) AppendRow(raw *common.RawKVEntry, } t.Length += 1 t.ApproximateSize += int64(len(raw.Key) + len(raw.Value) + len(raw.OldValue)) + t.Checksum = checksum return nil } diff --git a/pkg/common/event/mounter.go b/pkg/common/event/mounter.go index 38f9bb561e..776cd8df3b 100644 --- a/pkg/common/event/mounter.go +++ b/pkg/common/event/mounter.go @@ -20,14 +20,17 @@ import ( "time" "unsafe" - "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/spanz" "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/tablecodec" "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tidb/pkg/util/rowcodec" + "go.uber.org/zap" ) // DDLTableInfo contains the tableInfo about tidb_ddl_job and tidb_ddl_history @@ -49,43 +52,67 @@ type Mounter interface { // If the rawKV is a delete event, it will only decode the old value. // If the rawKV is an insert event, it will only decode the value. // If the rawKV is an update event, it will decode both the value and the old value. - DecodeToChunk(rawKV *common.RawKVEntry, tableInfo *common.TableInfo, chk *chunk.Chunk) (int, error) + DecodeToChunk(rawKV *common.RawKVEntry, tableInfo *common.TableInfo, chk *chunk.Chunk) (int, *integrity.Checksum, error) } type mounter struct { tz *time.Location + + integrity *integrity.Config + lastSkipOldValueTime time.Time } // NewMounter creates a mounter -func NewMounter(tz *time.Location) Mounter { +func NewMounter(tz *time.Location, integrity *integrity.Config) Mounter { return &mounter{ - tz: tz, + tz: tz, + integrity: integrity, } } // DecodeToChunk decodes the raw KV entry to a chunk, it returns the number of rows decoded. -func (m *mounter) DecodeToChunk(raw *common.RawKVEntry, tableInfo *common.TableInfo, chk *chunk.Chunk) (int, error) { +func (m *mounter) DecodeToChunk(raw *common.RawKVEntry, tableInfo *common.TableInfo, chk *chunk.Chunk) (int, *integrity.Checksum, error) { recordID, err := tablecodec.DecodeRowKey(raw.Key) if err != nil { - return 0, errors.Trace(err) + return 0, nil, errors.Trace(err) } if !bytes.HasPrefix(raw.Key, tablePrefix) { - return 0, nil + return 0, nil, nil } // key, physicalTableID, err := decodeTableID(raw.Key) // if err != nil { // return nil // } + var ( + decoder *rowcodec.ChunkDecoder + preChecksum uint32 + currentChecksum uint32 + matched bool + corrupted bool + ) + count := 0 if len(raw.OldValue) != 0 { if !rowcodec.IsNewFormat(raw.OldValue) { err = m.rawKVToChunkV1(raw.OldValue, tableInfo, chk, recordID) } else { - err = m.rawKVToChunkV2(raw.OldValue, tableInfo, chk, recordID) + decoder, err = m.rawKVToChunkV2(raw.OldValue, tableInfo, chk, recordID) } if err != nil { - return 0, errors.Trace(err) + return 0, nil, errors.Trace(err) + } + preChecksum, matched, err = m.verifyChecksum(tableInfo, chk.GetRow(count), raw.Key, recordID, decoder) + if err != nil { + return 0, nil, errors.Trace(err) + } + if !matched { + log.Error("previous columns checksum mismatch", + zap.Uint32("checksum", preChecksum), zap.Any("tableInfo", tableInfo)) + if m.integrity.ErrorHandle() { + return 0, nil, errors.ErrCorruptedDataMutation + } + corrupted = true } count++ } @@ -93,14 +120,39 @@ func (m *mounter) DecodeToChunk(raw *common.RawKVEntry, tableInfo *common.TableI if !rowcodec.IsNewFormat(raw.Value) { err = m.rawKVToChunkV1(raw.Value, tableInfo, chk, recordID) } else { - err = m.rawKVToChunkV2(raw.Value, tableInfo, chk, recordID) + decoder, err = m.rawKVToChunkV2(raw.Value, tableInfo, chk, recordID) } if err != nil { - return 0, errors.Trace(err) + return 0, nil, errors.Trace(err) + } + currentChecksum, matched, err = m.verifyChecksum(tableInfo, chk.GetRow(count), raw.Key, recordID, decoder) + if err != nil { + return 0, nil, errors.Trace(err) + } + if !matched { + log.Error("current columns checksum mismatch", + zap.Uint32("checksum", currentChecksum), zap.Any("tableInfo", tableInfo)) + if m.integrity.ErrorHandle() { + return 0, nil, errors.ErrCorruptedDataMutation + } + corrupted = true } count++ } - return count, nil + + var checksum *integrity.Checksum + // if both are 0, it means the checksum is not enabled + // so the checksum is nil to reduce memory allocation. + if preChecksum != 0 || currentChecksum != 0 { + checksum = &integrity.Checksum{ + Current: currentChecksum, + Previous: preChecksum, + Corrupted: corrupted, + Version: decoder.ChecksumVersion(), + } + } + + return count, checksum, nil } // IsLegacyFormatJob returns true if the job is from the legacy DDL list key. diff --git a/pkg/common/event/row_change.go b/pkg/common/event/row_change.go index 3448d3c52b..8d45008179 100644 --- a/pkg/common/event/row_change.go +++ b/pkg/common/event/row_change.go @@ -16,11 +16,11 @@ package event import ( "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/common/columnselector" + "github.com/pingcap/ticdc/pkg/integrity" timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/integrity" ) //go:generate msgp @@ -93,6 +93,8 @@ type RowEvent struct { Event RowChange ColumnSelector columnselector.Selector Callback func() + + Checksum *integrity.Checksum } func (e *RowEvent) IsDelete() bool { diff --git a/pkg/common/event/util.go b/pkg/common/event/util.go index cd355e0533..906b221851 100644 --- a/pkg/common/event/util.go +++ b/pkg/common/event/util.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/config" ticonfig "github.com/pingcap/tidb/pkg/config" tiddl "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/domain" @@ -69,7 +70,7 @@ func NewEventTestHelperWithTimeZone(t testing.TB, tz *time.Location) *EventTestH require.NoError(t, err) - mounter := NewMounter(tz) + mounter := NewMounter(tz, config.GetDefaultReplicaConfig().Integrity) return &EventTestHelper{ t: t, diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go index cb774bc897..35472abd6a 100644 --- a/pkg/config/replica_config.go +++ b/pkg/config/replica_config.go @@ -24,10 +24,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/config/outdated" - "github.com/pingcap/tiflow/pkg/integrity" "github.com/pingcap/tiflow/pkg/redo" "github.com/pingcap/tiflow/pkg/sink" "go.uber.org/zap" diff --git a/pkg/eventservice/event_broker.go b/pkg/eventservice/event_broker.go index 7a39555a46..1299ef8340 100644 --- a/pkg/eventservice/event_broker.go +++ b/pkg/eventservice/event_broker.go @@ -29,6 +29,7 @@ import ( appcontext "github.com/pingcap/ticdc/pkg/common/context" pevent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/messaging" "github.com/pingcap/ticdc/pkg/metrics" "github.com/pingcap/ticdc/pkg/node" @@ -109,6 +110,7 @@ func newEventBroker( schemaStore schemastore.SchemaStore, mc messaging.MessageSender, tz *time.Location, + integrity *integrity.Config, ) *eventBroker { // These numbers are define by real test result. // We noted that: @@ -131,7 +133,7 @@ func newEventBroker( tidbClusterID: id, eventStore: eventStore, pdClock: pdClock, - mounter: pevent.NewMounter(tz), + mounter: pevent.NewMounter(tz, integrity), schemaStore: schemaStore, changefeedMap: sync.Map{}, dispatchers: sync.Map{}, diff --git a/pkg/eventservice/event_broker_test.go b/pkg/eventservice/event_broker_test.go index 911b095085..b999c4ca47 100644 --- a/pkg/eventservice/event_broker_test.go +++ b/pkg/eventservice/event_broker_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/common/event" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/messaging" "github.com/pingcap/ticdc/pkg/node" "github.com/pingcap/ticdc/pkg/pdutil" @@ -44,7 +45,10 @@ func newEventBrokerForTest() (*eventBroker, *mockEventStore, *mockSchemaStore) { es := newMockEventStore(100) ss := newMockSchemaStore() mc := newMockMessageCenter() - return newEventBroker(context.Background(), 1, es, ss, mc, time.UTC), es, ss + return newEventBroker(context.Background(), 1, es, ss, mc, time.UTC, &integrity.Config{ + IntegrityCheckLevel: integrity.CheckLevelNone, + CorruptionHandleLevel: integrity.CorruptionHandleLevelWarn, + }), es, ss } func newMockDispatcherInfoForTest(t *testing.T) *mockDispatcherInfo { diff --git a/pkg/eventservice/event_service.go b/pkg/eventservice/event_service.go index 47a461167e..419c9be697 100644 --- a/pkg/eventservice/event_service.go +++ b/pkg/eventservice/event_service.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/ticdc/pkg/common" appcontext "github.com/pingcap/ticdc/pkg/common/context" "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/messaging" "go.uber.org/zap" ) @@ -49,6 +50,8 @@ type DispatcherInfo interface { IsOnlyReuse() bool GetBdrMode() bool + GetIntegrity() *integrity.Config + GetTimezone() *time.Location } // EventService accepts the requests of pulling events. @@ -62,7 +65,6 @@ type eventService struct { // TODO: use a better way to cache the acceptorInfos dispatcherInfo chan DispatcherInfo - tz *time.Location } func New(eventStore eventstore.EventStore, schemaStore schemastore.SchemaStore) common.SubModule { @@ -73,7 +75,6 @@ func New(eventStore eventstore.EventStore, schemaStore schemastore.SchemaStore) schemaStore: schemaStore, brokers: make(map[uint64]*eventBroker), dispatcherInfo: make(chan DispatcherInfo, basicChannelSize*16), - tz: time.Local, // FIXME use the timezone from the config } es.mc.RegisterHandler(messaging.EventServiceTopic, es.handleMessage) return es @@ -140,7 +141,7 @@ func (s *eventService) registerDispatcher(ctx context.Context, info DispatcherIn clusterID := info.GetClusterID() c, ok := s.brokers[clusterID] if !ok { - c = newEventBroker(ctx, clusterID, s.eventStore, s.schemaStore, s.mc, s.tz) + c = newEventBroker(ctx, clusterID, s.eventStore, s.schemaStore, s.mc, info.GetTimezone(), info.GetIntegrity()) s.brokers[clusterID] = c } c.addDispatcher(info) diff --git a/pkg/eventservice/event_service_test.go b/pkg/eventservice/event_service_test.go index 7dc3ca41d4..bbc196d784 100644 --- a/pkg/eventservice/event_service_test.go +++ b/pkg/eventservice/event_service_test.go @@ -32,6 +32,7 @@ import ( pevent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/messaging" "github.com/pingcap/ticdc/pkg/node" "github.com/pingcap/ticdc/pkg/pdutil" @@ -471,6 +472,8 @@ type mockDispatcherInfo struct { actionType eventpb.ActionType filter filter.Filter bdrMode bool + integrity *integrity.Config + tz *time.Location } func newMockDispatcherInfo(t *testing.T, dispatcherID common.DispatcherID, tableID int64, actionType eventpb.ActionType) *mockDispatcherInfo { @@ -490,6 +493,9 @@ func newMockDispatcherInfo(t *testing.T, dispatcherID common.DispatcherID, table startTs: 1, actionType: actionType, filter: filter, + bdrMode: false, + integrity: config.GetDefaultReplicaConfig().Integrity, + tz: time.Local, } } @@ -552,7 +558,15 @@ func (m *mockDispatcherInfo) IsOnlyReuse() bool { } func (m *mockDispatcherInfo) GetBdrMode() bool { - return false + return m.bdrMode +} + +func (m *mockDispatcherInfo) GetIntegrity() *integrity.Config { + return m.integrity +} + +func (m *mockDispatcherInfo) GetTimezone() *time.Location { + return m.tz } func genEvents(helper *pevent.EventTestHelper, t *testing.T, ddl string, dmls ...string) (pevent.DDLEvent, []*common.RawKVEntry) { diff --git a/pkg/integrity/checksum.go b/pkg/integrity/checksum.go new file mode 100644 index 0000000000..b956e8adde --- /dev/null +++ b/pkg/integrity/checksum.go @@ -0,0 +1,22 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package integrity + +// Checksum represent checksum for an RowChangedEvent +type Checksum struct { + Current uint32 + Previous uint32 + Corrupted bool + Version int +} diff --git a/pkg/integrity/integrity.go b/pkg/integrity/integrity.go new file mode 100644 index 0000000000..b7f54a6a12 --- /dev/null +++ b/pkg/integrity/integrity.go @@ -0,0 +1,71 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package integrity + +import ( + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/errors" + "go.uber.org/zap" +) + +// Config represents integrity check config for a changefeed. +type Config struct { + IntegrityCheckLevel string `toml:"integrity-check-level" json:"integrity-check-level"` + CorruptionHandleLevel string `toml:"corruption-handle-level" json:"corruption-handle-level"` +} + +const ( + // CheckLevelNone means no integrity check, the default value. + CheckLevelNone string = "none" + // CheckLevelCorrectness means check each row data correctness. + CheckLevelCorrectness string = "correctness" +) + +const ( + // CorruptionHandleLevelWarn is the default value, + // log the corrupted event, and mark it as corrupted and send it to the downstream. + CorruptionHandleLevelWarn string = "warn" + // CorruptionHandleLevelError means log the corrupted event, and then stopped the changefeed. + CorruptionHandleLevelError string = "error" +) + +// Validate the integrity config. +func (c *Config) Validate() error { + if c.IntegrityCheckLevel != CheckLevelNone && + c.IntegrityCheckLevel != CheckLevelCorrectness { + return errors.ErrInvalidReplicaConfig.GenWithStackByArgs() + } + if c.CorruptionHandleLevel != CorruptionHandleLevelWarn && + c.CorruptionHandleLevel != CorruptionHandleLevelError { + return errors.ErrInvalidReplicaConfig.GenWithStackByArgs() + } + + if c.Enabled() { + log.Info("integrity check is enabled", + zap.Any("integrityCheckLevel", c.IntegrityCheckLevel), + zap.Any("corruptionHandleLevel", c.CorruptionHandleLevel)) + } + + return nil +} + +// Enabled returns true if the integrity check is enabled. +func (c *Config) Enabled() bool { + return c.IntegrityCheckLevel == CheckLevelCorrectness +} + +// ErrorHandle returns true if the corruption handle level is error. +func (c *Config) ErrorHandle() bool { + return c.CorruptionHandleLevel == CorruptionHandleLevelError +} diff --git a/pkg/messaging/message.go b/pkg/messaging/message.go index 8adf01be77..cadd392690 100644 --- a/pkg/messaging/message.go +++ b/pkg/messaging/message.go @@ -25,7 +25,9 @@ import ( "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/node" + "github.com/pingcap/ticdc/pkg/util" "go.uber.org/zap" ) @@ -220,6 +222,25 @@ func (r RegisterDispatcherRequest) GetBdrMode() bool { return r.BdrMode } +func (r RegisterDispatcherRequest) GetIntegrity() *integrity.Config { + if r.RegisterDispatcherRequest.Integrity == nil { + return &integrity.Config{ + IntegrityCheckLevel: integrity.CheckLevelNone, + CorruptionHandleLevel: integrity.CorruptionHandleLevelWarn, + } + } + integrity := integrity.Config(*r.RegisterDispatcherRequest.Integrity) + return &integrity +} + +func (r RegisterDispatcherRequest) GetTimezone() *time.Location { + tz, err := util.GetTimezone(r.RegisterDispatcherRequest.GetTimezone()) + if err != nil { + log.Panic("Can't load time zone from dispatcher info", zap.Error(err)) + } + return tz +} + type IOTypeT interface { Unmarshal(data []byte) error Marshal() (data []byte, err error) diff --git a/pkg/sink/codec/avro/arvo.go b/pkg/sink/codec/avro/arvo.go index 445f2150ab..8aac16bf17 100644 --- a/pkg/sink/codec/avro/arvo.go +++ b/pkg/sink/codec/avro/arvo.go @@ -173,12 +173,11 @@ func (a *BatchEncoder) nativeValueWithExtension( native[tidbCommitTs] = int64(e.CommitTs) native[tidbPhysicalTime] = oracle.ExtractPhysical(e.CommitTs) - // TODO: EnableRowChecksum - // if a.config.EnableRowChecksum && e.Checksum != nil { - // native[tidbRowLevelChecksum] = strconv.FormatUint(uint64(e.Checksum.Current), 10) - // native[tidbCorrupted] = e.Checksum.Corrupted - // native[tidbChecksumVersion] = e.Checksum.Version - // } + if a.config.EnableRowChecksum && e.Checksum != nil { + native[tidbRowLevelChecksum] = strconv.FormatUint(uint64(e.Checksum.Current), 10) + native[tidbCorrupted] = e.Checksum.Corrupted + native[tidbChecksumVersion] = e.Checksum.Version + } return native } diff --git a/pkg/sink/codec/avro/decoder.go b/pkg/sink/codec/avro/decoder.go index 2db972d60b..f8a056d240 100644 --- a/pkg/sink/codec/avro/decoder.go +++ b/pkg/sink/codec/avro/decoder.go @@ -26,6 +26,7 @@ import ( commonType "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/sink/codec/common" timodel "github.com/pingcap/tidb/pkg/meta/model" pmodel "github.com/pingcap/tidb/pkg/parser/model" @@ -151,13 +152,12 @@ func (d *decoder) NextDMLEvent() (*commonEvent.DMLEvent, error) { if err != nil { return nil, errors.Trace(err) } - // TODO: Checksum - // corrupted := isCorrupted(valueMap) + corrupted := isCorrupted(valueMap) if found { - // event.Checksum = &integrity.Checksum{ - // Current: uint32(expectedChecksum), - // Corrupted: corrupted, - // } + event.Checksum = &integrity.Checksum{ + Current: uint32(expectedChecksum), + Corrupted: corrupted, + } } if isCorrupted(valueMap) { diff --git a/pkg/sink/codec/common/helper.go b/pkg/sink/codec/common/helper.go index e64627a88e..b2184a6a51 100644 --- a/pkg/sink/codec/common/helper.go +++ b/pkg/sink/codec/common/helper.go @@ -189,7 +189,7 @@ func MustQueryTimezone(ctx context.Context, db *sql.DB) string { } func queryRowChecksum( - ctx context.Context, db *sql.DB, event *commonEvent.RowChangedEvent, + ctx context.Context, db *sql.DB, event *commonEvent.DMLEvent, ) error { var ( schema = event.TableInfo.GetSchemaName() @@ -212,13 +212,19 @@ func queryRowChecksum( } defer conn.Close() + event.Rewind() + row, ok := event.GetNextRow() + if !ok { + log.Error("get RowChange failed") + } + columns := event.TableInfo.GetColumns() if event.Checksum.Current != 0 { - conditions := make(map[string]interface{}) + conditions := make(map[string]any) for _, name := range pkNames { - for _, col := range event.Columns { - colID := event.TableInfo.ForceGetColumnIDByName(col.Name) - if event.TableInfo.ForceGetColumnName(colID) == name { - conditions[name] = col.Value + for idx, col := range columns { + if col.Name.O == name { + d := row.Row.GetDatum(idx, &col.FieldType) + conditions[name] = d.GetValue() } } } @@ -232,12 +238,12 @@ func queryRowChecksum( } if event.Checksum.Previous != 0 { - conditions := make(map[string]interface{}) + conditions := make(map[string]any) for _, name := range pkNames { - for _, col := range event.PreColumns { - colID := event.TableInfo.ForceGetColumnIDByName(col.Name) - if event.TableInfo.ForceGetColumnName(colID) == name { - conditions[name] = col.Value + for idx, col := range columns { + if col.Name.O == name { + d := row.PreRow.GetDatum(idx, &col.FieldType) + conditions[name] = d.GetValue() } } } @@ -254,7 +260,7 @@ func queryRowChecksum( } func queryRowChecksumAux( - ctx context.Context, conn *sql.Conn, commitTs uint64, schema string, table string, conditions map[string]interface{}, + ctx context.Context, conn *sql.Conn, commitTs uint64, schema string, table string, conditions map[string]any, ) uint32 { var result uint32 // 1. set snapshot read diff --git a/pkg/sink/codec/common/utils.go b/pkg/sink/codec/common/utils.go index 37c0e86e19..8d48cffbb4 100644 --- a/pkg/sink/codec/common/utils.go +++ b/pkg/sink/codec/common/utils.go @@ -151,6 +151,7 @@ func NewLargeEvent4Test(t *testing.T) (*commonEvent.DDLEvent, *commonEvent.RowEv TableInfo: dmlEvent.TableInfo, Event: insert, ColumnSelector: columnselector.NewDefaultColumnSelector(), + Checksum: dmlEvent.Checksum, } update := commonEvent.RowChange{ PreRow: insert.Row, @@ -162,6 +163,7 @@ func NewLargeEvent4Test(t *testing.T) (*commonEvent.DDLEvent, *commonEvent.RowEv TableInfo: dmlEvent.TableInfo, Event: update, ColumnSelector: columnselector.NewDefaultColumnSelector(), + Checksum: dmlEvent.Checksum, } delete := commonEvent.RowChange{ PreRow: insert.Row, @@ -173,6 +175,7 @@ func NewLargeEvent4Test(t *testing.T) (*commonEvent.DDLEvent, *commonEvent.RowEv TableInfo: dmlEvent.TableInfo, Event: delete, ColumnSelector: columnselector.NewDefaultColumnSelector(), + Checksum: dmlEvent.Checksum, } ddlEvent := &commonEvent.DDLEvent{ diff --git a/pkg/sink/codec/common/verify_checksum.go b/pkg/sink/codec/common/verify_checksum.go index e928693c56..5eed6c4e60 100644 --- a/pkg/sink/codec/common/verify_checksum.go +++ b/pkg/sink/codec/common/verify_checksum.go @@ -20,44 +20,45 @@ import ( "fmt" "hash/crc32" "math" - "strconv" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" - "github.com/pingcap/ticdc/pkg/util" timodel "github.com/pingcap/tidb/pkg/meta/model" "github.com/pingcap/tidb/pkg/parser/mysql" - "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" "go.uber.org/zap" ) // VerifyChecksum calculate the checksum value, and compare it with the expected one, return error if not identical. -func VerifyChecksum(event *commonEvent.RowChangedEvent, db *sql.DB) error { +func VerifyChecksum(event *commonEvent.DMLEvent, db *sql.DB) error { // if expected is 0, it means the checksum is not enabled, so we don't need to verify it. // the data maybe restored by br, and the checksum is not enabled, so no expected here. + columns := event.TableInfo.GetColumns() + event.Rewind() + row, ok := event.GetNextRow() + if !ok { + log.Error("get RowChange failed") + } if event.Checksum.Current != 0 { - checksum, err := calculateChecksum(event.Columns, event.TableInfo.GetColumns()) + checksum, err := calculateChecksum(row.Row, columns) if err != nil { return errors.Trace(err) } if checksum != event.Checksum.Current { log.Error("current checksum mismatch", zap.Uint32("expected", event.Checksum.Current), zap.Uint32("actual", checksum), zap.Any("event", event)) - for _, col := range event.Columns { - colID := event.TableInfo.ForceGetColumnIDByName(col.Name) - colInfo := event.TableInfo.ForceGetColumnInfo(colID) + for _, col := range columns { log.Info("data corrupted, print each column for debugging", - zap.String("name", colInfo.Name.O), zap.Any("type", colInfo.GetType()), - zap.Any("charset", colInfo.GetCharset()), zap.Any("flag", colInfo.GetFlag()), - zap.Any("value", col.Value), zap.Any("default", colInfo.GetDefaultValue())) + zap.String("name", col.Name.O), zap.Any("type", col.GetType()), + zap.Any("charset", col.GetCharset()), zap.Any("flag", col.GetFlag()), + zap.Any("default", col.GetDefaultValue())) } return fmt.Errorf("current checksum mismatch, current: %d, expected: %d", checksum, event.Checksum.Current) } } if event.Checksum.Previous != 0 { - checksum, err := calculateChecksum(event.PreColumns, event.TableInfo.GetColumns()) + checksum, err := calculateChecksum(row.PreRow, columns) if err != nil { return errors.Trace(err) } @@ -65,13 +66,11 @@ func VerifyChecksum(event *commonEvent.RowChangedEvent, db *sql.DB) error { log.Error("previous checksum mismatch", zap.Uint32("expected", event.Checksum.Previous), zap.Uint32("actual", checksum), zap.Any("event", event)) - for _, col := range event.PreColumns { - colID := event.TableInfo.ForceGetColumnIDByName(col.Name) - colInfo := event.TableInfo.ForceGetColumnInfo(colID) + for _, col := range columns { log.Info("data corrupted, print each column for debugging", - zap.String("name", colInfo.Name.O), zap.Any("type", colInfo.GetType()), - zap.Any("charset", colInfo.GetCharset()), zap.Any("flag", colInfo.GetFlag()), - zap.Any("value", col.Value), zap.Any("default", colInfo.GetDefaultValue())) + zap.String("name", col.Name.O), zap.Any("type", col.GetType()), + zap.Any("charset", col.GetCharset()), zap.Any("flag", col.GetFlag()), + zap.Any("default", col.GetDefaultValue())) } return fmt.Errorf("previous checksum mismatch, current: %d, expected: %d", checksum, event.Checksum.Previous) } @@ -86,17 +85,18 @@ func VerifyChecksum(event *commonEvent.RowChangedEvent, db *sql.DB) error { // calculate the checksum, caller should make sure all columns is ordered by the column's id. // by follow: https://github.com/pingcap/tidb/blob/e3417913f58cdd5a136259b902bf177eaf3aa637/util/rowcodec/common.go#L294 -func calculateChecksum(columns []*common.Column, columnInfo []*timodel.ColumnInfo) (uint32, error) { +func calculateChecksum(row chunk.Row, columnInfo []*timodel.ColumnInfo) (uint32, error) { var ( checksum uint32 err error ) buf := make([]byte, 0) - for idx, col := range columns { + for idx, col := range columnInfo { if len(buf) > 0 { buf = buf[:0] } - buf, err = buildChecksumBytes(buf, col.Value, columnInfo[idx].GetType()) + // buf = row.GetRaw(idx) + buf, err = buildChecksumBytes(buf, row, idx, col) if err != nil { return 0, errors.Trace(err) } @@ -107,121 +107,37 @@ func calculateChecksum(columns []*common.Column, columnInfo []*timodel.ColumnInf // buildChecksumBytes append value to the buf, mysqlType is used to convert value interface to concrete type. // by follow: https://github.com/pingcap/tidb/blob/e3417913f58cdd5a136259b902bf177eaf3aa637/util/rowcodec/common.go#L308 -func buildChecksumBytes(buf []byte, value interface{}, mysqlType byte) ([]byte, error) { - if value == nil { +func buildChecksumBytes(buf []byte, row chunk.Row, idx int, col *timodel.ColumnInfo) ([]byte, error) { + if row.IsNull(idx) { return buf, nil } - - switch mysqlType { + d := row.GetDatum(idx, &col.FieldType) + switch col.GetType() { // TypeTiny, TypeShort, TypeInt32 is encoded as int32 // TypeLong is encoded as int32 if signed, else int64. // TypeLongLong is encoded as int64 if signed, else uint64, // if bigintUnsignedHandlingMode set as string, encode as string. case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeInt24, mysql.TypeYear: - var ( - v uint64 - err error - ) - switch a := value.(type) { - case int32: - v = uint64(a) - case uint32: - v = uint64(a) - case int64: - v = uint64(a) - case uint64: - v = a - case string: - v, err = strconv.ParseUint(a, 10, 64) - if err != nil { - return nil, errors.Trace(err) - } - case map[string]interface{}: - // this may only happen for bigint larger than math.uint64 - v = uint64(a["value"].(int64)) - default: - log.Panic("unknown golang type for the integral value", - zap.Any("value", value), zap.Any("mysqlType", mysqlType)) - } - buf = binary.LittleEndian.AppendUint64(buf, v) - // TypeFloat encoded as float32, TypeDouble encoded as float64 + buf = binary.LittleEndian.AppendUint64(buf, d.GetUint64()) case mysql.TypeFloat, mysql.TypeDouble: - var v float64 - switch a := value.(type) { - case float32: - v = float64(a) - case float64: - v = a - } + v := d.GetFloat64() if math.IsInf(v, 0) || math.IsNaN(v) { v = 0 } buf = binary.LittleEndian.AppendUint64(buf, math.Float64bits(v)) - // TypeEnum, TypeSet encoded as string - // but convert to int by the getColumnValue function - case mysql.TypeEnum, mysql.TypeSet: - var number uint64 - switch v := value.(type) { - case uint64: - number = v - case int64: - number = uint64(v) - } - buf = binary.LittleEndian.AppendUint64(buf, number) + case mysql.TypeEnum: + buf = binary.LittleEndian.AppendUint64(buf, d.GetMysqlEnum().Value) + case mysql.TypeSet: + buf = binary.LittleEndian.AppendUint64(buf, d.GetMysqlSet().Value) case mysql.TypeBit: - var number uint64 - switch v := value.(type) { - // TypeBit encoded as bytes for the avro protocol - case []byte: - number = MustBinaryLiteralToInt(v) - // TypeBit encoded as uint64 for the simple protocol - case uint64: - number = v - } + number := MustBinaryLiteralToInt(row.GetBytes(idx)) buf = binary.LittleEndian.AppendUint64(buf, number) - // encoded as bytes if binary flag set to true, else string - case mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: - switch a := value.(type) { - case string: - buf = appendLengthValue(buf, []byte(a)) - case []byte: - buf = appendLengthValue(buf, a) - default: - log.Panic("unknown golang type for the string value", - zap.Any("value", value), zap.Any("mysqlType", mysqlType)) - } - case mysql.TypeTimestamp: - location := "Local" - var ts string - switch data := value.(type) { - case map[string]interface{}: - ts = data["value"].(string) - location = data["location"].(string) - case string: - ts = data - } - ts, err := util.ConvertTimezone(ts, location) - if err != nil { - log.Panic("convert timestamp to timezone failed", - zap.String("timestamp", ts), zap.String("location", location), - zap.Error(err)) - } - buf = appendLengthValue(buf, []byte(ts)) - // all encoded as string - case mysql.TypeDatetime, mysql.TypeDate, mysql.TypeDuration, mysql.TypeNewDate: - buf = appendLengthValue(buf, []byte(value.(string))) - // encoded as string if decimalHandlingMode set to string, it's required to enable checksum. - case mysql.TypeNewDecimal: - buf = appendLengthValue(buf, []byte(value.(string))) - // encoded as string - case mysql.TypeJSON: - buf = appendLengthValue(buf, []byte(value.(string))) - // this should not happen, does not take into the checksum calculation. + case mysql.TypeVarchar, mysql.TypeVarString, mysql.TypeString, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob, + mysql.TypeDatetime, mysql.TypeDate, mysql.TypeTimestamp, mysql.TypeNewDate, mysql.TypeDuration, + mysql.TypeNewDecimal, mysql.TypeJSON, mysql.TypeTiDBVectorFloat32: + buf = appendLengthValue(buf, UnsafeStringToBytes(fmt.Sprintf("%v", d.GetValue()))) case mysql.TypeNull, mysql.TypeGeometry: // do nothing - case mysql.TypeTiDBVectorFloat32: - vec, _ := types.ParseVectorFloat32(value.(string)) - buf = vec.SerializeTo(buf) default: return buf, errors.New("invalid type for the checksum calculation") } diff --git a/pkg/sink/codec/debezium/debezium_test.go b/pkg/sink/codec/debezium/debezium_test.go index 59fc557ef0..c15de176c9 100644 --- a/pkg/sink/codec/debezium/debezium_test.go +++ b/pkg/sink/codec/debezium/debezium_test.go @@ -47,7 +47,7 @@ func NewSQLTestHelper(t *testing.T, tableName, initialCreateTableDDL string) *SQ job := helper.DDL2Job(initialCreateTableDDL) require.NotNil(t, job) - mounter := commonEvent.NewMounter(time.UTC) + mounter := commonEvent.NewMounter(time.UTC, config.GetDefaultReplicaConfig().Integrity) tableInfo := helper.GetTableInfo(job) diff --git a/pkg/sink/codec/simple/avro.go b/pkg/sink/codec/simple/avro.go index 8f2b02b38b..708ae95fc8 100644 --- a/pkg/sink/codec/simple/avro.go +++ b/pkg/sink/codec/simple/avro.go @@ -270,19 +270,18 @@ func (a *avroMarshaller) newDMLMessageMap( } } - // TODO: EnableRowChecksum - // if a.config.EnableRowChecksum && event.Checksum != nil { - // cc := map[string]interface{}{ - // "version": event.Checksum.Version, - // "corrupted": event.Checksum.Corrupted, - // "current": int64(event.Checksum.Current), - // "previous": int64(event.Checksum.Previous), - // } - - // holder := genericMapPool.Get().(map[string]interface{}) - // holder["com.pingcap.simple.avro.Checksum"] = cc - // dmlMessagePayload["checksum"] = holder - // } + if a.config.EnableRowChecksum && event.Checksum != nil { + cc := map[string]interface{}{ + "version": event.Checksum.Version, + "corrupted": event.Checksum.Corrupted, + "current": int64(event.Checksum.Current), + "previous": int64(event.Checksum.Previous), + } + + holder := genericMapPool.Get().(map[string]interface{}) + holder["com.pingcap.simple.avro.Checksum"] = cc + dmlMessagePayload["checksum"] = holder + } if event.IsInsert() { data := a.collectColumns(event.GetRows(), event.TableInfo, onlyHandleKey, event.ColumnSelector) diff --git a/pkg/sink/codec/simple/decoder.go b/pkg/sink/codec/simple/decoder.go index 06a2d6895b..60ae2a4484 100644 --- a/pkg/sink/codec/simple/decoder.go +++ b/pkg/sink/codec/simple/decoder.go @@ -27,6 +27,7 @@ import ( commonType "github.com/pingcap/ticdc/pkg/common" commonEvent "github.com/pingcap/ticdc/pkg/common/event" "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/integrity" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/tidb/br/pkg/storage" timodel "github.com/pingcap/tidb/pkg/meta/model" @@ -606,23 +607,22 @@ func buildDMLEvent(msg *message, tableInfo *commonType.TableInfo, enableRowCheck } result.Rows = chk - // TODO: enableRowChecksum - // if enableRowChecksum && msg.Checksum != nil { - // result.Checksum = &integrity.Checksum{ - // Current: msg.Checksum.Current, - // Previous: msg.Checksum.Previous, - // Corrupted: msg.Checksum.Corrupted, - // Version: msg.Checksum.Version, - // } - - // err := common.VerifyChecksum(result, db) - // if err != nil || msg.Checksum.Corrupted { - // log.Warn("consumer detect checksum corrupted", - // zap.String("schema", msg.Schema), zap.String("table", msg.Table), zap.Error(err)) - // return nil, cerror.ErrDecodeFailed.GenWithStackByArgs("checksum corrupted") - - // } - // } + if enableRowChecksum && msg.Checksum != nil { + result.Checksum = &integrity.Checksum{ + Current: msg.Checksum.Current, + Previous: msg.Checksum.Previous, + Corrupted: msg.Checksum.Corrupted, + Version: msg.Checksum.Version, + } + + err := common.VerifyChecksum(result, db) + if err != nil || msg.Checksum.Corrupted { + log.Warn("consumer detect checksum corrupted", + zap.String("schema", msg.Schema), zap.String("table", msg.Table), zap.Error(err)) + return nil, errors.ErrDecodeFailed.GenWithStackByArgs("checksum corrupted") + + } + } return result, nil } diff --git a/pkg/sink/codec/simple/message.go b/pkg/sink/codec/simple/message.go index 8617f4f9f6..155abccacf 100644 --- a/pkg/sink/codec/simple/message.go +++ b/pkg/sink/codec/simple/message.go @@ -343,15 +343,14 @@ func (a *JSONMarshaller) newDMLMessage( m.Data = a.formatColumns(event.GetRows(), event.TableInfo, onlyHandleKey, event.ColumnSelector) m.Old = a.formatColumns(event.GetPreRows(), event.TableInfo, onlyHandleKey, event.ColumnSelector) } - // TODO: EnableRowChecksum - // if a.config.EnableRowChecksum && event.Checksum != nil { - // m.Checksum = &checksum{ - // Version: event.Checksum.Version, - // Corrupted: event.Checksum.Corrupted, - // Current: event.Checksum.Current, - // Previous: event.Checksum.Previous, - // } - // } + if a.config.EnableRowChecksum && event.Checksum != nil { + m.Checksum = &checksum{ + Version: event.Checksum.Version, + Corrupted: event.Checksum.Corrupted, + Current: event.Checksum.Current, + Previous: event.Checksum.Previous, + } + } return m }