diff --git a/br/pkg/lightning/backend/kv/sql2kv.go b/br/pkg/lightning/backend/kv/sql2kv.go index fef655348c4a1..4f077b1e1a756 100644 --- a/br/pkg/lightning/backend/kv/sql2kv.go +++ b/br/pkg/lightning/backend/kv/sql2kv.go @@ -153,6 +153,21 @@ type Pairs struct { MemBuf *MemBuf } +// GroupedPairs is a map from index ID to KvPairs. +type GroupedPairs map[int64][]common.KvPair + +// SplitIntoChunks implements the encode.Rows interface. It just satisfies the +// type system and should never be called. +func (GroupedPairs) SplitIntoChunks(int) []encode.Rows { + panic("not implemented") +} + +// Clear implements the encode.Rows interface. It just satisfies the type system +// and should never be called. +func (GroupedPairs) Clear() encode.Rows { + panic("not implemented") +} + // MakeRowsFromKvPairs converts a KvPair slice into a Rows instance. This is // mainly used for testing only. The resulting Rows instance should only be used // for the importer backend. @@ -171,7 +186,21 @@ func MakeRowFromKvPairs(pairs []common.KvPair) encode.Row { // back into a slice of KvPair. This method panics if the Rows is not // constructed in such way. func Rows2KvPairs(rows encode.Rows) []common.KvPair { - return rows.(*Pairs).Pairs + switch v := rows.(type) { + case *Pairs: + return v.Pairs + case GroupedPairs: + cnt := 0 + for _, pairs := range v { + cnt += len(pairs) + } + res := make([]common.KvPair, 0, cnt) + for _, pairs := range v { + res = append(res, pairs...) + } + return res + } + panic(fmt.Sprintf("unknown Rows type %T", rows)) } // Row2KvPairs converts a Row instance constructed from MakeRowFromKvPairs diff --git a/br/pkg/lightning/backend/local/BUILD.bazel b/br/pkg/lightning/backend/local/BUILD.bazel index 4777cc55dc981..f032a6a629c49 100644 --- a/br/pkg/lightning/backend/local/BUILD.bazel +++ b/br/pkg/lightning/backend/local/BUILD.bazel @@ -50,11 +50,9 @@ go_library( "//pkg/kv", "//pkg/metrics", "//pkg/parser/model", - "//pkg/parser/mysql", "//pkg/sessionctx/variable", "//pkg/table", "//pkg/tablecodec", - "//pkg/types", "//pkg/util", "//pkg/util/codec", "//pkg/util/compress", @@ -76,7 +74,6 @@ go_library( "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_kvproto//pkg/pdpb", "@com_github_pingcap_tipb//go-tipb", - "@com_github_tikv_client_go_v2//error", "@com_github_tikv_client_go_v2//kv", "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//tikv", diff --git a/br/pkg/lightning/backend/local/duplicate.go b/br/pkg/lightning/backend/local/duplicate.go index a9a655e8dd4fb..6cd78cd254934 100644 --- a/br/pkg/lightning/backend/local/duplicate.go +++ b/br/pkg/lightning/backend/local/duplicate.go @@ -21,7 +21,6 @@ import ( "fmt" "io" "math" - "slices" "sync" "github.com/cockroachdb/pebble" @@ -43,20 +42,16 @@ import ( "github.com/pingcap/tidb/pkg/distsql" tidbkv "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" - "github.com/pingcap/tidb/pkg/parser/mysql" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/tablecodec" - "github.com/pingcap/tidb/pkg/types" "github.com/pingcap/tidb/pkg/util/codec" "github.com/pingcap/tidb/pkg/util/hack" "github.com/pingcap/tidb/pkg/util/ranger" - tikverror "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/tikv" kvutil "github.com/tikv/client-go/v2/util" "go.uber.org/atomic" "go.uber.org/zap" "golang.org/x/sync/errgroup" - "golang.org/x/time/rate" ) const ( @@ -1004,29 +999,14 @@ func (local *DupeController) ResolveDuplicateRows(ctx context.Context, tbl table }() switch algorithm { - case config.DupeResAlgRecord, config.DupeResAlgNone: + case config.DupeResAlgNone: logger.Warn("skipping resolution due to selected algorithm. this table will become inconsistent!", zap.String("category", "resolve-dupe"), zap.Stringer("algorithm", algorithm)) return nil - case config.DupeResAlgRemove: - case config.DupeResAlgReplace: + case config.DupeResAlgReplace, config.DupeResAlgErr: default: panic(fmt.Sprintf("[resolve-dupe] unknown resolution algorithm %v", algorithm)) } - // TODO: reuse the *kv.SessionOptions from NewEncoder for picking the correct time zone. - decoder, err := kv.NewTableKVDecoder(tbl, tableName, &encode.SessionOptions{ - SQLMode: mysql.ModeStrictAllTables, - }, log.FromContext(ctx)) - if err != nil { - return errors.Trace(err) - } - - tableIDs := physicalTableIDs(tbl.Meta()) - keyInTable := func(key []byte) bool { - return slices.Contains(tableIDs, tablecodec.DecodeTableID(key)) - } - - errLimiter := rate.NewLimiter(1, 1) pool := utils.NewWorkerPool(uint(local.dupeConcurrency), "resolve duplicate rows") tblInfo, err := json.Marshal(tbl.Meta()) @@ -1037,31 +1017,6 @@ func (local *DupeController) ResolveDuplicateRows(ctx context.Context, tbl table zap.ByteString("tblInfo", tblInfo)) switch algorithm { - case config.DupeResAlgRemove: - err = local.errorMgr.RemoveAllConflictKeys( - ctx, tableName, pool, - func(ctx context.Context, handleRows [][2][]byte) error { - for { - err := local.deleteDuplicateRows(ctx, logger, handleRows, decoder, keyInTable) - if err == nil { - return nil - } - if types.ErrBadNumber.Equal(err) { - logger.Warn("delete duplicate rows encounter error", log.ShortError(err)) - return common.ErrResolveDuplicateRows.Wrap(errors.Trace(err)).GenWithStackByArgs(tableName) - } - if log.IsContextCanceledError(err) { - return errors.Trace(err) - } - if !tikverror.IsErrWriteConflict(errors.Cause(err)) { - logger.Warn("delete duplicate rows encounter error", log.ShortError(errors.Trace(err))) - } - if err = errLimiter.Wait(ctx); err != nil { - return errors.Trace(err) - } - } - }, - ) case config.DupeResAlgReplace: err = local.errorMgr.ReplaceConflictKeys( ctx, tbl, tableName, pool, @@ -1075,12 +1030,16 @@ func (local *DupeController) ResolveDuplicateRows(ctx context.Context, tbl table func(ctx context.Context, key []byte) error { err := local.deleteDuplicateRow(ctx, logger, key) if err != nil { - logger.Debug("delete duplicate rows encounter error", zap.Error(err)) - return errors.Trace(err) + logger.Warn("delete duplicate rows encounter error", log.ShortError(err)) + return common.ErrResolveDuplicateRows.Wrap(errors.Trace(err)).GenWithStackByArgs(tableName) } return nil }, ) + case config.DupeResAlgErr: + err = local.errorMgr.ResolveConflictKeysError( + ctx, tableName, + ) } return errors.Trace(err) @@ -1130,63 +1089,3 @@ func (local *DupeController) deleteDuplicateRow( return errors.Trace(err) } - -func (local *DupeController) deleteDuplicateRows( - ctx context.Context, - logger *log.Task, - handleRows [][2][]byte, - decoder *kv.TableKVDecoder, - keyInTable func(key []byte) bool, -) (err error) { - // Starts a Delete transaction. - txn, err := local.tikvCli.Begin() - if err != nil { - return errors.Trace(err) - } - defer func() { - if err == nil { - err = txn.Commit(ctx) - } else { - if rollbackErr := txn.Rollback(); rollbackErr != nil { - logger.Warn("failed to rollback transaction", zap.Error(rollbackErr)) - } - } - }() - - deleteKey := func(key []byte) error { - logger.Debug("will delete key", zap.String("category", "resolve-dupe"), logutil.Key("key", key)) - return txn.Delete(key) - } - - // Collect all rows & index keys into the deletion transaction. - // (if the number of duplicates is small this should fit entirely in memory) - // (Txn's MemBuf's bufferSizeLimit is currently infinity) - for _, handleRow := range handleRows { - // Skip the row key if it's not in the table. - // This can happen if the table has been recreated or truncated, - // and the duplicate key is from the old table. - if !keyInTable(handleRow[0]) { - continue - } - logger.Debug("found row to resolve", zap.String("category", "resolve-dupe"), - logutil.Key("handle", handleRow[0]), - logutil.Key("row", handleRow[1])) - - if err := deleteKey(handleRow[0]); err != nil { - return errors.Trace(err) - } - - handle, err := decoder.DecodeHandleFromRowKey(handleRow[0]) - if err != nil { - return errors.Trace(err) - } - - err = decoder.IterRawIndexKeys(handle, handleRow[1], deleteKey) - if err != nil { - return errors.Trace(err) - } - } - - logger.Debug("number of KV pairs to be deleted", zap.String("category", "resolve-dupe"), zap.Int("count", txn.Len())) - return nil -} diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 68816897609f7..e2d5d711c6d03 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -459,13 +459,12 @@ func (c *BackendConfig) adjust() { // Backend is a local backend. type Backend struct { - pdCli pd.Client - pdHTTPCli pdhttp.Client - splitCli split.SplitClient - tikvCli *tikvclient.KVStore - tls *common.TLS - regionSizeGetter TableRegionSizeGetter - tikvCodec tikvclient.Codec + pdCli pd.Client + pdHTTPCli pdhttp.Client + splitCli split.SplitClient + tikvCli *tikvclient.KVStore + tls *common.TLS + tikvCodec tikvclient.Codec BackendConfig engineMgr *engineManager @@ -500,10 +499,17 @@ func NewBackend( ctx context.Context, tls *common.TLS, config BackendConfig, - regionSizeGetter TableRegionSizeGetter, + pdSvcDiscovery pd.ServiceDiscovery, ) (b *Backend, err error) { config.adjust() - pdAddrs := strings.Split(config.PDAddr, ",") + var pdAddrs []string + if pdSvcDiscovery != nil { + pdAddrs = pdSvcDiscovery.GetServiceURLs() + // TODO(lance6716): if PD client can support creating a client with external + // service discovery, we can directly pass pdSvcDiscovery. + } else { + pdAddrs = strings.Split(config.PDAddr, ",") + } pdCli, err := pd.NewClientWithContext( ctx, pdAddrs, tls.ToPDSecurityOption(), pd.WithGRPCDialOptions(maxCallMsgSize...), @@ -552,13 +558,12 @@ func NewBackend( writeLimiter = noopStoreWriteLimiter{} } local := &Backend{ - pdCli: pdCli, - pdHTTPCli: pdHTTPCli, - splitCli: splitCli, - tikvCli: tikvCli, - tls: tls, - regionSizeGetter: regionSizeGetter, - tikvCodec: tikvCodec, + pdCli: pdCli, + pdHTTPCli: pdHTTPCli, + splitCli: splitCli, + tikvCli: tikvCli, + tls: tls, + tikvCodec: tikvCodec, BackendConfig: config, diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index eb7ef76763968..452bb9844fd96 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -17,7 +17,6 @@ package local import ( "bytes" "context" - "database/sql" "math" "slices" "sort" @@ -62,51 +61,6 @@ var ( splitRetryTimes = 8 ) -// TableRegionSizeGetter get table region size. -type TableRegionSizeGetter interface { - GetTableRegionSize(ctx context.Context, tableID int64) (map[uint64]int64, error) -} - -// TableRegionSizeGetterImpl implements TableRegionSizeGetter. -type TableRegionSizeGetterImpl struct { - DB *sql.DB -} - -var _ TableRegionSizeGetter = &TableRegionSizeGetterImpl{} - -// GetTableRegionSize implements TableRegionSizeGetter. -func (g *TableRegionSizeGetterImpl) GetTableRegionSize(ctx context.Context, tableID int64) (map[uint64]int64, error) { - if g.DB == nil { - return nil, errors.Errorf("db is nil") - } - exec := &common.SQLWithRetry{ - DB: g.DB, - Logger: log.FromContext(ctx), - } - - stats := make(map[uint64]int64) - err := exec.Transact(ctx, "fetch region approximate sizes", func(ctx context.Context, tx *sql.Tx) error { - rows, err := tx.QueryContext(ctx, "SELECT REGION_ID, APPROXIMATE_SIZE FROM information_schema.TIKV_REGION_STATUS WHERE TABLE_ID = ?", tableID) - if err != nil { - return errors.Trace(err) - } - //nolint: errcheck - defer rows.Close() - var ( - regionID uint64 - size int64 - ) - for rows.Next() { - if err = rows.Scan(®ionID, &size); err != nil { - return errors.Trace(err) - } - stats[regionID] = size * units.MiB - } - return rows.Err() - }) - return stats, errors.Trace(err) -} - // SplitAndScatterRegionInBatches splits&scatter regions in batches. // Too many split&scatter requests may put a lot of pressure on TiKV and PD. func (local *Backend) SplitAndScatterRegionInBatches( diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index 117576e244db0..b36bff11615cf 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -462,9 +462,8 @@ func doTestBatchSplitRegionByRanges(ctx context.Context, t *testing.T, hook clie keys := [][]byte{[]byte(""), []byte("aay"), []byte("bba"), []byte("bbh"), []byte("cca"), []byte("")} client := initTestSplitClient(keys, hook) local := &Backend{ - splitCli: client, - regionSizeGetter: &TableRegionSizeGetterImpl{}, - logger: log.L(), + splitCli: client, + logger: log.L(), } local.RegionSplitBatchSize = 4 local.RegionSplitConcurrency = 4 @@ -721,9 +720,8 @@ func TestSplitAndScatterRegionInBatches(t *testing.T) { keys := [][]byte{[]byte(""), []byte("a"), []byte("b"), []byte("")} client := initTestSplitClient(keys, nil) local := &Backend{ - splitCli: client, - regionSizeGetter: &TableRegionSizeGetterImpl{}, - logger: log.L(), + splitCli: client, + logger: log.L(), } local.RegionSplitBatchSize = 4 local.RegionSplitConcurrency = 4 @@ -807,9 +805,8 @@ func doTestBatchSplitByRangesWithClusteredIndex(t *testing.T, hook clientHook) { keys = append(keys, tableEndKey, []byte("")) client := initTestSplitClient(keys, hook) local := &Backend{ - splitCli: client, - regionSizeGetter: &TableRegionSizeGetterImpl{}, - logger: log.L(), + splitCli: client, + logger: log.L(), } local.RegionSplitBatchSize = 10 local.RegionSplitConcurrency = 10 diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index 25ba1c277472f..4773a564fdc1c 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -307,7 +307,7 @@ func NewTiDBBackend( onDuplicate = config.ErrorOnDup } default: - log.FromContext(ctx).Warn("unsupported conflict strategy, overwrite with `error`") + log.FromContext(ctx).Warn("unsupported conflict strategy for TiDB backend, overwrite with `error`") onDuplicate = config.ErrorOnDup } return &tidbBackend{ diff --git a/br/pkg/lightning/common/BUILD.bazel b/br/pkg/lightning/common/BUILD.bazel index 6fdc16688a869..86cc2f65c5cf8 100644 --- a/br/pkg/lightning/common/BUILD.bazel +++ b/br/pkg/lightning/common/BUILD.bazel @@ -112,7 +112,7 @@ go_test( ], embed = [":common"], flaky = True, - shard_count = 29, + shard_count = 28, deps = [ "//br/pkg/errors", "//br/pkg/lightning/log", diff --git a/br/pkg/lightning/common/util.go b/br/pkg/lightning/common/util.go index e5c9cffd46f59..7ebbcd4e62a3d 100644 --- a/br/pkg/lightning/common/util.go +++ b/br/pkg/lightning/common/util.go @@ -454,13 +454,16 @@ func KillMySelf() error { return errors.Trace(err) } -// KvPair is a pair of key and value. +// KvPair contains a key-value pair and other fields that can be used to ingest +// KV pairs into TiKV. type KvPair struct { // Key is the key of the KV pair Key []byte // Val is the value of the KV pair Val []byte - // RowID is the row id of the KV pair. + // RowID identifies a KvPair in case two KvPairs are equal in Key and Val. It's + // often set to the file offset of the KvPair in the source file or the record + // handle. RowID []byte } @@ -482,19 +485,6 @@ func TableHasAutoID(info *model.TableInfo) bool { return TableHasAutoRowID(info) || info.GetAutoIncrementColInfo() != nil || info.ContainsAutoRandomBits() } -// StringSliceEqual checks if two string slices are equal. -func StringSliceEqual(a, b []string) bool { - if len(a) != len(b) { - return false - } - for i, v := range a { - if v != b[i] { - return false - } - } - return true -} - // GetAutoRandomColumn return the column with auto_random, return nil if the table doesn't have it. // todo: better put in ddl package, but this will cause import cycle since ddl package import lightning func GetAutoRandomColumn(tblInfo *model.TableInfo) *model.ColumnInfo { diff --git a/br/pkg/lightning/common/util_test.go b/br/pkg/lightning/common/util_test.go index 11b126a2ebdd8..774319d7300a7 100644 --- a/br/pkg/lightning/common/util_test.go +++ b/br/pkg/lightning/common/util_test.go @@ -163,19 +163,6 @@ func TestSQLWithRetry(t *testing.T) { require.Nil(t, mock.ExpectationsWereMet()) } -func TestStringSliceEqual(t *testing.T) { - assert.True(t, common.StringSliceEqual(nil, nil)) - assert.True(t, common.StringSliceEqual(nil, []string{})) - assert.False(t, common.StringSliceEqual(nil, []string{"a"})) - assert.False(t, common.StringSliceEqual([]string{"a"}, nil)) - assert.True(t, common.StringSliceEqual([]string{"a"}, []string{"a"})) - assert.False(t, common.StringSliceEqual([]string{"a"}, []string{"b"})) - assert.True(t, common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b", "c"})) - assert.False(t, common.StringSliceEqual([]string{"a"}, []string{"a", "b", "c"})) - assert.False(t, common.StringSliceEqual([]string{"a", "b", "c"}, []string{"a", "b"})) - assert.False(t, common.StringSliceEqual([]string{"a", "x", "y"}, []string{"a", "y", "x"})) -} - func TestInterpolateMySQLString(t *testing.T) { assert.Equal(t, "'123'", common.InterpolateMySQLString("123")) assert.Equal(t, "'1''23'", common.InterpolateMySQLString("1'23")) diff --git a/br/pkg/lightning/config/BUILD.bazel b/br/pkg/lightning/config/BUILD.bazel index 1ae65a73dfd5e..68b51db7b3efd 100644 --- a/br/pkg/lightning/config/BUILD.bazel +++ b/br/pkg/lightning/config/BUILD.bazel @@ -44,7 +44,6 @@ go_test( flaky = True, shard_count = 48, deps = [ - "//br/pkg/lightning/common", "@com_github_burntsushi_toml//:toml", "@com_github_stretchr_testify//require", ], diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index c8c97f274b460..8aa6b3ecd3dfd 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -63,11 +63,11 @@ const ( // CheckpointDriverFile is a constant for choosing the "File" checkpoint driver in the configuration. CheckpointDriverFile = "file" - // ReplaceOnDup indicates using REPLACE INTO to insert data + // ReplaceOnDup indicates using REPLACE INTO to insert data for TiDB backend. ReplaceOnDup = "replace" - // IgnoreOnDup indicates using INSERT IGNORE INTO to insert data + // IgnoreOnDup indicates using INSERT IGNORE INTO to insert data for TiDB backend. IgnoreOnDup = "ignore" - // ErrorOnDup indicates using INSERT INTO to insert data, which would violate PK or UNIQUE constraint + // ErrorOnDup indicates using INSERT INTO to insert data, which would violate PK or UNIQUE constraint for TiDB backend. ErrorOnDup = "error" // KVWriteBatchSize batch size when write to TiKV. @@ -600,21 +600,13 @@ const ( // DupeResAlgNone doesn't detect duplicate. DupeResAlgNone DuplicateResolutionAlgorithm = iota - // DupeResAlgRecord only records duplicate records to `lightning_task_info.conflict_error_v2` table on the target TiDB. - DupeResAlgRecord - - // DupeResAlgRemove records all duplicate records like the 'record' algorithm and remove all information related to the - // duplicated rows. Users need to analyze the lightning_task_info.conflict_error_v2 table to add back the correct rows. - DupeResAlgRemove - // DupeResAlgReplace records all duplicate records like the 'record' algorithm, and remove some rows with conflict // and reserve other rows that can be kept and not cause conflict anymore. Users need to analyze the // lightning_task_info.conflict_error_v2 table to check whether the reserved data cater to their need and check whether // they need to add back the correct rows. DupeResAlgReplace - // DupeResAlgErr reports an error and stops the import process. - // Note: this value is only used for internal. + // DupeResAlgErr reports an error after detecting the first conflict and stops the import process. DupeResAlgErr ) @@ -623,7 +615,7 @@ func (dra *DuplicateResolutionAlgorithm) UnmarshalTOML(v any) error { if val, ok := v.(string); ok { return dra.FromStringValue(val) } - return errors.Errorf("invalid duplicate-resolution '%v', please choose valid option between ['record', 'none', 'remove']", v) + return errors.Errorf("invalid duplicate-resolution '%v', please choose valid option between ['none', 'replace', 'error']", v) } // MarshalText implements the encoding.TextMarshaler interface. @@ -634,16 +626,14 @@ func (dra DuplicateResolutionAlgorithm) MarshalText() ([]byte, error) { // FromStringValue parses the string value to the DuplicateResolutionAlgorithm. func (dra *DuplicateResolutionAlgorithm) FromStringValue(s string) error { switch strings.ToLower(s) { - case "record": - *dra = DupeResAlgRecord case "none": *dra = DupeResAlgNone - case "remove": - *dra = DupeResAlgRemove case "replace": *dra = DupeResAlgReplace + case "error": + *dra = DupeResAlgErr default: - return errors.Errorf("invalid duplicate-resolution '%s', please choose valid option between ['record', 'none', 'remove']", s) + return errors.Errorf("invalid duplicate-resolution '%s', please choose valid option between ['none', 'replace', 'error']", s) } return nil } @@ -661,14 +651,12 @@ func (dra *DuplicateResolutionAlgorithm) UnmarshalJSON(data []byte) error { // String implements the fmt.Stringer interface. func (dra DuplicateResolutionAlgorithm) String() string { switch dra { - case DupeResAlgRecord: - return "record" case DupeResAlgNone: return "none" - case DupeResAlgRemove: - return "remove" case DupeResAlgReplace: return "replace" + case DupeResAlgErr: + return "error" default: panic(fmt.Sprintf("invalid duplicate-resolution type '%d'", dra)) } diff --git a/br/pkg/lightning/config/config_test.go b/br/pkg/lightning/config/config_test.go index d7047b406c6aa..07f77abf37178 100644 --- a/br/pkg/lightning/config/config_test.go +++ b/br/pkg/lightning/config/config_test.go @@ -31,7 +31,6 @@ import ( "time" "github.com/BurntSushi/toml" - "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/stretchr/testify/require" ) @@ -738,19 +737,16 @@ func TestDurationMarshalJSON(t *testing.T) { func TestDuplicateResolutionAlgorithm(t *testing.T) { var dra DuplicateResolutionAlgorithm - require.NoError(t, dra.FromStringValue("record")) - require.Equal(t, DupeResAlgRecord, dra) require.NoError(t, dra.FromStringValue("none")) require.Equal(t, DupeResAlgNone, dra) - require.NoError(t, dra.FromStringValue("remove")) - require.Equal(t, DupeResAlgRemove, dra) require.NoError(t, dra.FromStringValue("replace")) require.Equal(t, DupeResAlgReplace, dra) + require.NoError(t, dra.FromStringValue("error")) + require.Equal(t, DupeResAlgErr, dra) - require.Equal(t, "record", DupeResAlgRecord.String()) require.Equal(t, "none", DupeResAlgNone.String()) - require.Equal(t, "remove", DupeResAlgRemove.String()) require.Equal(t, "replace", DupeResAlgReplace.String()) + require.Equal(t, "error", DupeResAlgErr.String()) } func TestLoadConfig(t *testing.T) { @@ -996,14 +992,14 @@ func TestAdjustConflictStrategy(t *testing.T) { cfg.TikvImporter.Backend = BackendLocal cfg.Conflict.Strategy = ReplaceOnDup cfg.TikvImporter.ParallelImport = false - cfg.TikvImporter.DuplicateResolution = DupeResAlgRemove + cfg.TikvImporter.DuplicateResolution = DupeResAlgReplace require.ErrorContains(t, cfg.Adjust(ctx), "conflict.strategy cannot be used with tikv-importer.duplicate-resolution") cfg.TikvImporter.Backend = BackendLocal cfg.Conflict.Strategy = "" cfg.TikvImporter.OnDuplicate = ReplaceOnDup cfg.TikvImporter.ParallelImport = false - cfg.TikvImporter.DuplicateResolution = DupeResAlgRemove + cfg.TikvImporter.DuplicateResolution = DupeResAlgReplace require.ErrorContains(t, cfg.Adjust(ctx), "tikv-importer.on-duplicate cannot be used with tikv-importer.duplicate-resolution") } @@ -1221,35 +1217,22 @@ func TestCreateSeveralConfigsWithDifferentFilters(t *testing.T) { [mydumper] filter = ["db1.tbl1", "db2.*", "!db2.tbl1"] `))) - require.Equal(t, 3, len(cfg1.Mydumper.Filter)) - require.True(t, common.StringSliceEqual( - cfg1.Mydumper.Filter, - []string{"db1.tbl1", "db2.*", "!db2.tbl1"}, - )) - require.True(t, common.StringSliceEqual(GetDefaultFilter(), originalDefaultCfg)) + require.Equal(t, []string{"db1.tbl1", "db2.*", "!db2.tbl1"}, cfg1.Mydumper.Filter) + require.Equal(t, GetDefaultFilter(), originalDefaultCfg) cfg2 := NewConfig() - require.True(t, common.StringSliceEqual( - cfg2.Mydumper.Filter, - originalDefaultCfg, - )) - require.True(t, common.StringSliceEqual(GetDefaultFilter(), originalDefaultCfg)) + require.Equal(t, originalDefaultCfg, cfg2.Mydumper.Filter) + require.Equal(t, GetDefaultFilter(), originalDefaultCfg) gCfg1, err := LoadGlobalConfig([]string{"-f", "db1.tbl1", "-f", "db2.*", "-f", "!db2.tbl1"}, nil) require.NoError(t, err) - require.True(t, common.StringSliceEqual( - gCfg1.Mydumper.Filter, - []string{"db1.tbl1", "db2.*", "!db2.tbl1"}, - )) - require.True(t, common.StringSliceEqual(GetDefaultFilter(), originalDefaultCfg)) + require.Equal(t, []string{"db1.tbl1", "db2.*", "!db2.tbl1"}, gCfg1.Mydumper.Filter) + require.Equal(t, GetDefaultFilter(), originalDefaultCfg) gCfg2, err := LoadGlobalConfig([]string{}, nil) require.NoError(t, err) - require.True(t, common.StringSliceEqual( - gCfg2.Mydumper.Filter, - originalDefaultCfg, - )) - require.True(t, common.StringSliceEqual(GetDefaultFilter(), originalDefaultCfg)) + require.Equal(t, originalDefaultCfg, gCfg2.Mydumper.Filter) + require.Equal(t, GetDefaultFilter(), originalDefaultCfg) } func TestCompressionType(t *testing.T) { diff --git a/br/pkg/lightning/errormanager/BUILD.bazel b/br/pkg/lightning/errormanager/BUILD.bazel index 291befefb1a8b..0c37f9aea9882 100644 --- a/br/pkg/lightning/errormanager/BUILD.bazel +++ b/br/pkg/lightning/errormanager/BUILD.bazel @@ -44,6 +44,7 @@ go_test( deps = [ "//br/pkg/lightning/backend/encode", "//br/pkg/lightning/backend/kv", + "//br/pkg/lightning/common", "//br/pkg/lightning/config", "//br/pkg/lightning/log", "//br/pkg/utils", diff --git a/br/pkg/lightning/errormanager/errormanager.go b/br/pkg/lightning/errormanager/errormanager.go index 4cc4a8e49447a..62c65da824a09 100644 --- a/br/pkg/lightning/errormanager/errormanager.go +++ b/br/pkg/lightning/errormanager/errormanager.go @@ -19,9 +19,7 @@ import ( "context" "database/sql" "fmt" - "math" "strings" - "sync" "github.com/jedib0t/go-pretty/v6/table" "github.com/jedib0t/go-pretty/v6/text" @@ -138,13 +136,6 @@ const ( sqlValuesConflictErrorIndex = "(?,?,?,?,?,?,?,?,?,?)" - selectConflictKeysRemove = ` - SELECT _tidb_rowid, raw_handle, raw_row - FROM %s.` + ConflictErrorTableName + ` - WHERE table_name = ? AND _tidb_rowid >= ? and _tidb_rowid < ? - ORDER BY _tidb_rowid LIMIT ?; - ` - selectIndexConflictKeysReplace = ` SELECT raw_key, index_name, raw_value, raw_handle FROM %s.` + ConflictErrorTableName + ` @@ -159,6 +150,24 @@ const ( ORDER BY raw_key; ` + deleteNullDataRow = ` + DELETE FROM %s.` + ConflictErrorTableName + ` + WHERE key_data = "" and row_data = ""; + ` + + selectConflictKeysCountError = ` + SELECT COUNT(*) + FROM %s.` + ConflictErrorTableName + ` + WHERE table_name = ?; + ` + + selectConflictKeysError = ` + SELECT raw_key, raw_row + FROM %s.` + ConflictErrorTableName + ` + WHERE table_name = ? + LIMIT 1; + ` + insertIntoDupRecord = ` INSERT INTO %s.` + DupRecordTable + ` (task_id, table_name, path, offset, error, row_id, row_data) @@ -459,86 +468,6 @@ func (em *ErrorManager) RecordIndexConflictError( return gerr } -// RemoveAllConflictKeys query all conflicting rows (handle and their -// values) from the current error report and resolve them concurrently by removing all of them. -func (em *ErrorManager) RemoveAllConflictKeys( - ctx context.Context, - tableName string, - pool *utils.WorkerPool, - fn func(ctx context.Context, handleRows [][2][]byte) error, -) error { - if em.db == nil { - return nil - } - - const rowLimit = 1000 - taskCh := make(chan [2]int64) - taskWg := &sync.WaitGroup{} - g, gCtx := errgroup.WithContext(ctx) - - go func() { - //nolint:staticcheck - //lint:ignore SA2000 - taskWg.Add(1) - taskCh <- [2]int64{0, math.MaxInt64} - taskWg.Wait() - close(taskCh) - }() - - for t := range taskCh { - start, end := t[0], t[1] - pool.ApplyOnErrorGroup(g, func() error { - defer taskWg.Done() - - var handleRows [][2][]byte - for start < end { - rows, err := em.db.QueryContext( - gCtx, common.SprintfWithIdentifiers(selectConflictKeysRemove, em.schema), - tableName, start, end, rowLimit) - if err != nil { - return errors.Trace(err) - } - var lastRowID int64 - for rows.Next() { - var handleRow [2][]byte - if err := rows.Scan(&lastRowID, &handleRow[0], &handleRow[1]); err != nil { - return errors.Trace(err) - } - handleRows = append(handleRows, handleRow) - } - if err := rows.Err(); err != nil { - return errors.Trace(err) - } - if err := rows.Close(); err != nil { - return errors.Trace(err) - } - if len(handleRows) == 0 { - break - } - if err := fn(gCtx, handleRows); err != nil { - return errors.Trace(err) - } - start = lastRowID + 1 - // If the remaining tasks cannot be processed at once, split the task - // into two subtasks and send one of them to the other idle worker if possible. - if end-start > rowLimit { - mid := start + (end-start)/2 - taskWg.Add(1) - select { - case taskCh <- [2]int64{mid, end}: - end = mid - default: - taskWg.Done() - } - } - handleRows = handleRows[:0] - } - return nil - }) - } - return errors.Trace(g.Wait()) -} - // ReplaceConflictKeys query all conflicting rows (handle and their // values) from the current error report and resolve them // by replacing the necessary rows and reserving the others. @@ -568,6 +497,13 @@ func (em *ErrorManager) ReplaceConflictKeys( } g, gCtx := errgroup.WithContext(ctx) + + exec := common.SQLWithRetry{ + DB: em.db, + Logger: em.logger, + HideQueryLog: redact.NeedRedact(), + } + pool.ApplyOnErrorGroup(g, func() error { // TODO: provide a detailed document to explain the algorithm and link it here // demo for "replace" algorithm: https://github.com/lyzx2001/tidb-conflict-replace @@ -638,12 +574,6 @@ func (em *ErrorManager) ReplaceConflictKeys( // find out all the KV pairs that are contained in the data KV kvPairs := encoder.SessionCtx.TakeKvPairs() - exec := common.SQLWithRetry{ - DB: em.db, - Logger: em.logger, - HideQueryLog: redact.NeedRedact(), - } - for _, kvPair := range kvPairs.Pairs { em.logger.Debug("got encoded KV", logutil.Key("key", kvPair.Key), @@ -674,7 +604,7 @@ func (em *ErrorManager) ReplaceConflictKeys( sb := &strings.Builder{} _, err2 := common.FprintfWithIdentifiers(sb, insertIntoConflictErrorData, em.schema) if err2 != nil { - return err2 + return errors.Trace(err2) } var sqlArgs []any sb.WriteString(sqlValuesConflictErrorData) @@ -688,9 +618,9 @@ func (em *ErrorManager) ReplaceConflictKeys( 1, ) _, err := txn.ExecContext(c, sb.String(), sqlArgs...) - return err + return errors.Trace(err) }); err != nil { - return err + return errors.Trace(err) } if err := fnDeleteKey(gCtx, rawHandle); err != nil { return errors.Trace(err) @@ -819,12 +749,84 @@ func (em *ErrorManager) ReplaceConflictKeys( return errors.Trace(err) } + // delete the additionally inserted rows for nonclustered PK + if err := exec.Transact(ctx, "delete additionally inserted rows for conflict detection 'replace' mode", + func(c context.Context, txn *sql.Tx) error { + sb := &strings.Builder{} + _, err2 := common.FprintfWithIdentifiers(sb, deleteNullDataRow, em.schema) + if err2 != nil { + return errors.Trace(err2) + } + _, err := txn.ExecContext(c, sb.String()) + return errors.Trace(err) + }); err != nil { + return errors.Trace(err) + } + return nil }) return errors.Trace(g.Wait()) } +// ResolveConflictKeysError query all conflicting rows (handle and their +// values) from the current error report and return error +// if the number of the conflicting rows is larger than 0. +func (em *ErrorManager) ResolveConflictKeysError( + ctx context.Context, + tableName string, +) error { + if em.db == nil { + return nil + } + + _, gCtx := errgroup.WithContext(ctx) + + kvRows, err := em.db.QueryContext( + gCtx, common.SprintfWithIdentifiers(selectConflictKeysCountError, em.schema), + tableName) + if err != nil { + return errors.Trace(err) + } + defer kvRows.Close() + var kvRowsCount int64 + for kvRows.Next() { + if err := kvRows.Scan(&kvRowsCount); err != nil { + return errors.Trace(err) + } + } + if err := kvRows.Err(); err != nil { + return errors.Trace(err) + } + + em.logger.Debug("got kv rows count from table", + zap.Int64("kv rows count", kvRowsCount)) + if kvRowsCount > 0 { + rows, err := em.db.QueryContext( + gCtx, common.SprintfWithIdentifiers(selectConflictKeysError, em.schema), + tableName) + if err != nil { + return errors.Trace(err) + } + defer rows.Close() + + var rawKey, rawRow []byte + for rows.Next() { + if err := rows.Scan(&rawKey, &rawRow); err != nil { + return errors.Trace(err) + } + em.logger.Debug("got raw_key, raw_row from table", + logutil.Key("raw_key", rawKey), + zap.Binary("raw_row", rawRow)) + } + if err := rows.Err(); err != nil { + return errors.Trace(err) + } + return common.ErrFoundDuplicateKeys.FastGenByArgs(rawKey, rawRow) + } + return nil +} + // RecordDuplicateCount reduce the counter of "duplicate entry" errors. // Currently, the count will not be shared for multiple lightning instances. func (em *ErrorManager) RecordDuplicateCount(cnt int64) error { @@ -975,9 +977,9 @@ func (em *ErrorManager) LogErrorDetails() { } if errCnt := em.conflictError(); errCnt > 0 { if em.conflictV1Enabled { - em.logger.Warn(fmtErrMsg(errCnt, "data conflict", ConflictErrorTableName)) + em.logger.Warn(fmtErrMsg(errCnt, "conflict", ConflictErrorTableName)) } else { - em.logger.Warn(fmtErrMsg(errCnt, "data conflict", DupRecordTable)) + em.logger.Warn(fmtErrMsg(errCnt, "conflict", DupRecordTable)) } } } diff --git a/br/pkg/lightning/errormanager/errormanager_test.go b/br/pkg/lightning/errormanager/errormanager_test.go index 5745cbfb5de70..eac0b4546bfd9 100644 --- a/br/pkg/lightning/errormanager/errormanager_test.go +++ b/br/pkg/lightning/errormanager/errormanager_test.go @@ -17,12 +17,9 @@ package errormanager import ( "bytes" "context" - "database/sql" "database/sql/driver" "fmt" "io" - "math/rand" - "strconv" "testing" "github.com/DATA-DOG/go-sqlmock" @@ -157,35 +154,6 @@ func (c mockConn) QueryContext(_ context.Context, query string, args []driver.Na return &mockRows{start: start, end: end}, nil } -func TestRemoveAllConflictKeys(t *testing.T) { - const totalRows = int64(1 << 18) - driverName := "errmgr-mock-" + strconv.Itoa(rand.Int()) - sql.Register(driverName, mockDriver{totalRows: totalRows}) - db, err := sql.Open(driverName, "") - require.NoError(t, err) - defer db.Close() - - cfg := config.NewConfig() - cfg.TikvImporter.DuplicateResolution = config.DupeResAlgRemove - cfg.App.TaskInfoSchemaName = "lightning_errors" - em := New(db, cfg, log.L()) - ctx := context.Background() - err = em.Init(ctx) - require.NoError(t, err) - - resolved := atomic.NewInt64(0) - pool := utils.NewWorkerPool(16, "resolve duplicate rows by remove") - err = em.RemoveAllConflictKeys( - ctx, "test", pool, - func(ctx context.Context, handleRows [][2][]byte) error { - resolved.Add(int64(len(handleRows))) - return nil - }, - ) - require.NoError(t, err) - require.Equal(t, totalRows, resolved.Load()) -} - func TestReplaceConflictOneKey(t *testing.T) { column1 := &model.ColumnInfo{ ID: 1, @@ -322,6 +290,10 @@ func TestReplaceConflictOneKey(t *testing.T) { WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). AddRow(data1RowKey, data1RowValue). AddRow(data1RowKey, data2RowValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -521,6 +493,10 @@ func TestReplaceConflictOneUniqueKey(t *testing.T) { WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). AddRow(data1RowKey, data1RowValue). AddRow(data1RowKey, data3RowValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace diff --git a/br/pkg/lightning/errormanager/resolveconflict_test.go b/br/pkg/lightning/errormanager/resolveconflict_test.go index d7e7581deb335..dd90aebb3ba88 100644 --- a/br/pkg/lightning/errormanager/resolveconflict_test.go +++ b/br/pkg/lightning/errormanager/resolveconflict_test.go @@ -24,6 +24,7 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/pingcap/tidb/br/pkg/lightning/backend/encode" tidbkv "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" + "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/errormanager" "github.com/pingcap/tidb/br/pkg/lightning/log" @@ -189,6 +190,10 @@ func TestReplaceConflictMultipleKeysNonclusteredPk(t *testing.T) { WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). AddRow(data2NonclusteredKey, data2NonclusteredValue). AddRow(data6NonclusteredKey, data6NonclusteredValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -351,6 +356,10 @@ func TestReplaceConflictOneKeyNonclusteredPk(t *testing.T) { mockDB.ExpectQuery("\\QSELECT raw_key, raw_value FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key\\E"). WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). AddRow(data4RowKey, data4RowValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -529,6 +538,10 @@ func TestReplaceConflictOneUniqueKeyNonclusteredPk(t *testing.T) { AddRow(data5RowKey, data5RowValue). AddRow(data2RowKey, data2RowValue). AddRow(data4RowKey, data4RowValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -726,6 +739,10 @@ func TestReplaceConflictOneUniqueKeyNonclusteredVarcharPk(t *testing.T) { AddRow(data5RowKey, data5RowValue). AddRow(data2RowKey, data2RowValue). AddRow(data4RowKey, data4RowValue)) + mockDB.ExpectBegin() + mockDB.ExpectExec("DELETE FROM `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -792,3 +809,91 @@ func TestReplaceConflictOneUniqueKeyNonclusteredVarcharPk(t *testing.T) { err = mockDB.ExpectationsWereMet() require.NoError(t, err) } + +func TestResolveConflictKeysError(t *testing.T) { + p := parser.New() + node, _, err := p.ParseSQL("create table a (a varchar(20) primary key clustered, b int not null, c text, key uni_b(b));") + require.NoError(t, err) + mockSctx := mock.NewContext() + info, err := ddl.MockTableInfo(mockSctx, node[0].(*ast.CreateTableStmt), 108) + require.NoError(t, err) + info.State = model.StatePublic + tbl, err := tables.TableFromMeta(tidbkv.NewPanickingAllocators(0), info) + require.NoError(t, err) + + sessionOpts := encode.SessionOptions{ + SQLMode: mysql.ModeStrictAllTables, + Timestamp: 1234567890, + } + + encoder, err := tidbkv.NewBaseKVEncoder(&encode.EncodingConfig{ + Table: tbl, + SessionOptions: sessionOpts, + Logger: log.L(), + }) + require.NoError(t, err) + encoder.SessionCtx.GetSessionVars().RowEncoder.Enable = true + + data1 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(6), + types.NewStringDatum("1.csv"), + types.NewIntDatum(1), + } + data2 := []types.Datum{ + types.NewIntDatum(1), + types.NewIntDatum(6), + types.NewStringDatum("2.csv"), + types.NewIntDatum(2), + } + data3 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + types.NewIntDatum(3), + } + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data1) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data2) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data3) + require.NoError(t, err) + kvPairs := encoder.SessionCtx.TakeKvPairs() + + data1RowKey := kvPairs.Pairs[0].Key + data1RowValue := kvPairs.Pairs[0].Val + + db, mockDB, err := sqlmock.New() + require.NoError(t, err) + defer func() { + _ = db.Close() + }() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + mockDB.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_task_info`"). + WillReturnResult(sqlmock.NewResult(1, 1)) + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). + WillReturnResult(sqlmock.NewResult(2, 1)) + mockDB.ExpectQuery("\\QSELECT COUNT(*) FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ?\\E"). + WillReturnRows(sqlmock.NewRows([]string{"COUNT(*)"}). + AddRow(2)) + mockDB.ExpectQuery("\\QSELECT raw_key, raw_row FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? LIMIT 1\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value"}). + AddRow(data1RowKey, data1RowValue)) + + cfg := config.NewConfig() + cfg.TikvImporter.DuplicateResolution = config.DupeResAlgErr + cfg.App.TaskInfoSchemaName = "lightning_task_info" + em := errormanager.New(db, cfg, log.L()) + err = em.Init(ctx) + require.NoError(t, err) + + err = em.ResolveConflictKeysError( + ctx, "a", + ) + require.Error(t, err, common.ErrFoundDuplicateKeys) + err = mockDB.ExpectationsWereMet() + require.NoError(t, err) +} diff --git a/br/pkg/lightning/importer/import.go b/br/pkg/lightning/importer/import.go index 7389915c7bbd7..3660b7e4acddc 100644 --- a/br/pkg/lightning/importer/import.go +++ b/br/pkg/lightning/importer/import.go @@ -386,9 +386,6 @@ func NewImportControllerWithPauser( initGlobalConfig(tls.ToTiKVSecurityConfig()) encodingBuilder = local.NewEncodingBuilder(ctx) - regionSizeGetter := &local.TableRegionSizeGetterImpl{ - DB: db, - } // get resource group name. exec := common.SQLWithRetry{ @@ -419,7 +416,7 @@ func NewImportControllerWithPauser( raftKV2SwitchModeDuration = cfg.Cron.SwitchMode.Duration } backendConfig := local.NewBackendConfig(cfg, maxOpenFiles, p.KeyspaceName, p.ResourceGroupName, p.TaskType, raftKV2SwitchModeDuration) - backendObj, err = local.NewBackend(ctx, tls, backendConfig, regionSizeGetter) + backendObj, err = local.NewBackend(ctx, tls, backendConfig, pdCli.GetServiceDiscovery()) if err != nil { return nil, common.NormalizeOrWrapErr(common.ErrUnknown, err) } @@ -1611,20 +1608,26 @@ func (rc *Controller) importTables(ctx context.Context) (finalErr error) { } // Disable GC because TiDB enables GC already. - - currentLeaderAddr := rc.pdCli.GetLeaderAddr() + urlsWithScheme := rc.pdCli.GetServiceDiscovery().GetServiceURLs() // remove URL scheme - currentLeaderAddr = strings.TrimPrefix(currentLeaderAddr, "http://") - currentLeaderAddr = strings.TrimPrefix(currentLeaderAddr, "https://") + urlsWithoutScheme := make([]string, 0, len(urlsWithScheme)) + for _, u := range urlsWithScheme { + u = strings.TrimPrefix(u, "http://") + u = strings.TrimPrefix(u, "https://") + urlsWithoutScheme = append(urlsWithoutScheme, u) + } kvStore, err = driver.TiKVDriver{}.OpenWithOptions( - fmt.Sprintf("tikv://%s?disableGC=true&keyspaceName=%s", currentLeaderAddr, rc.keyspaceName), + fmt.Sprintf( + "tikv://%s?disableGC=true&keyspaceName=%s", + strings.Join(urlsWithoutScheme, ","), rc.keyspaceName, + ), driver.WithSecurity(rc.tls.ToTiKVSecurityConfig()), ) if err != nil { return errors.Trace(err) } etcdCli, err := clientv3.New(clientv3.Config{ - Endpoints: []string{rc.cfg.TiDB.PdAddr}, + Endpoints: urlsWithScheme, AutoSyncInterval: 30 * time.Second, TLS: rc.tls.TLSConfig(), }) @@ -1837,7 +1840,7 @@ func (rc *Controller) importTables(ctx context.Context) (finalErr error) { } func (rc *Controller) registerTaskToPD(ctx context.Context) (undo func(), _ error) { - etcdCli, err := dialEtcdWithCfg(ctx, rc.cfg, rc.pdCli.GetLeaderAddr()) + etcdCli, err := dialEtcdWithCfg(ctx, rc.cfg, rc.pdCli.GetServiceDiscovery().GetServiceURLs()) if err != nil { return nil, errors.Trace(err) } @@ -2138,6 +2141,7 @@ func (rc *Controller) preCheckRequirements(ctx context.Context) error { rc.status.TotalFileSize.Store(estimatedSizeResult.SizeWithoutIndex) } if isLocalBackend(rc.cfg) { + // TODO(lance6716): use rc.pdCli to create pd controller pdController, err := pdutil.NewPdController(ctx, rc.pdCli.GetLeaderAddr(), rc.tls.TLSConfig(), rc.tls.ToPDSecurityOption()) if err != nil { diff --git a/br/pkg/lightning/importer/precheck.go b/br/pkg/lightning/importer/precheck.go index 5023d1b6b785d..fd17d01850c79 100644 --- a/br/pkg/lightning/importer/precheck.go +++ b/br/pkg/lightning/importer/precheck.go @@ -23,11 +23,11 @@ func WithPrecheckKey(ctx context.Context, key precheckContextKey, val any) conte // PrecheckItemBuilder is used to build precheck items type PrecheckItemBuilder struct { - cfg *config.Config - dbMetas []*mydump.MDDatabaseMeta - preInfoGetter PreImportInfoGetter - checkpointsDB checkpoints.DB - pdLeaderAddrGetter func(context.Context) string + cfg *config.Config + dbMetas []*mydump.MDDatabaseMeta + preInfoGetter PreImportInfoGetter + checkpointsDB checkpoints.DB + pdAddrsGetter func(context.Context) []string } // NewPrecheckItemBuilderFromConfig creates a new PrecheckItemBuilder from config @@ -88,29 +88,29 @@ func NewPrecheckItemBuilder( checkpointsDB checkpoints.DB, pdHTTPCli pdhttp.Client, ) *PrecheckItemBuilder { - leaderAddrGetter := func(context.Context) string { - return cfg.TiDB.PdAddr + pdAddrsGetter := func(context.Context) []string { + return []string{cfg.TiDB.PdAddr} } // in tests we may not have a pdCli if pdHTTPCli != nil { - leaderAddrGetter = func(ctx context.Context) string { + pdAddrsGetter = func(ctx context.Context) []string { leaderInfo, err := pdHTTPCli.GetLeader(ctx) if err != nil { - return cfg.TiDB.PdAddr + return []string{cfg.TiDB.PdAddr} } addrs := leaderInfo.GetClientUrls() if len(addrs) == 0 { - return cfg.TiDB.PdAddr + return []string{cfg.TiDB.PdAddr} } - return addrs[0] + return addrs } } return &PrecheckItemBuilder{ - cfg: cfg, - dbMetas: dbMetas, - preInfoGetter: preInfoGetter, - checkpointsDB: checkpointsDB, - pdLeaderAddrGetter: leaderAddrGetter, + cfg: cfg, + dbMetas: dbMetas, + preInfoGetter: preInfoGetter, + checkpointsDB: checkpointsDB, + pdAddrsGetter: pdAddrsGetter, } } @@ -142,7 +142,7 @@ func (b *PrecheckItemBuilder) BuildPrecheckItem(checkID precheck.CheckItemID) (p case precheck.CheckLocalTempKVDir: return NewLocalTempKVDirCheckItem(b.cfg, b.preInfoGetter, b.dbMetas), nil case precheck.CheckTargetUsingCDCPITR: - return NewCDCPITRCheckItem(b.cfg, b.pdLeaderAddrGetter), nil + return NewCDCPITRCheckItem(b.cfg, b.pdAddrsGetter), nil default: return nil, errors.Errorf("unsupported check item: %v", checkID) } diff --git a/br/pkg/lightning/importer/precheck_impl.go b/br/pkg/lightning/importer/precheck_impl.go index 1f24d6a8cf293..bf9920b19f50a 100644 --- a/br/pkg/lightning/importer/precheck_impl.go +++ b/br/pkg/lightning/importer/precheck_impl.go @@ -752,19 +752,19 @@ func (ci *checkpointCheckItem) checkpointIsValid(ctx context.Context, tableInfo // CDCPITRCheckItem check downstream has enabled CDC or PiTR. It's exposed to let // caller override the Instruction message. type CDCPITRCheckItem struct { - cfg *config.Config - Instruction string - leaderAddrGetter func(context.Context) string + cfg *config.Config + Instruction string + pdAddrsGetter func(context.Context) []string // used in test etcdCli *clientv3.Client } // NewCDCPITRCheckItem creates a checker to check downstream has enabled CDC or PiTR. -func NewCDCPITRCheckItem(cfg *config.Config, leaderAddrGetter func(context.Context) string) precheck.Checker { +func NewCDCPITRCheckItem(cfg *config.Config, pdAddrsGetter func(context.Context) []string) precheck.Checker { return &CDCPITRCheckItem{ - cfg: cfg, - Instruction: "local backend is not compatible with them. Please switch to tidb backend then try again.", - leaderAddrGetter: leaderAddrGetter, + cfg: cfg, + Instruction: "local backend is not compatible with them. Please switch to tidb backend then try again.", + pdAddrsGetter: pdAddrsGetter, } } @@ -776,7 +776,7 @@ func (*CDCPITRCheckItem) GetCheckItemID() precheck.CheckItemID { func dialEtcdWithCfg( ctx context.Context, cfg *config.Config, - leaderAddr string, + addrs []string, ) (*clientv3.Client, error) { cfg2, err := cfg.ToTLS() if err != nil { @@ -786,7 +786,7 @@ func dialEtcdWithCfg( return clientv3.New(clientv3.Config{ TLS: tlsConfig, - Endpoints: []string{leaderAddr}, + Endpoints: addrs, AutoSyncInterval: 30 * time.Second, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ @@ -813,7 +813,7 @@ func (ci *CDCPITRCheckItem) Check(ctx context.Context) (*precheck.CheckResult, e if ci.etcdCli == nil { var err error - ci.etcdCli, err = dialEtcdWithCfg(ctx, ci.cfg, ci.leaderAddrGetter(ctx)) + ci.etcdCli, err = dialEtcdWithCfg(ctx, ci.cfg, ci.pdAddrsGetter(ctx)) if err != nil { return nil, errors.Trace(err) } diff --git a/br/pkg/lightning/verification/BUILD.bazel b/br/pkg/lightning/verification/BUILD.bazel index 122e4b0e8535f..a28cd05043f0f 100644 --- a/br/pkg/lightning/verification/BUILD.bazel +++ b/br/pkg/lightning/verification/BUILD.bazel @@ -17,9 +17,11 @@ go_test( timeout = "short", srcs = ["checksum_test.go"], flaky = True, + shard_count = 3, deps = [ ":verification", "//br/pkg/lightning/common", "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//tikv", ], ) diff --git a/br/pkg/lightning/verification/checksum.go b/br/pkg/lightning/verification/checksum.go index d1219455d3254..d30e143b301ad 100644 --- a/br/pkg/lightning/verification/checksum.go +++ b/br/pkg/lightning/verification/checksum.go @@ -25,7 +25,8 @@ import ( var ecmaTable = crc64.MakeTable(crc64.ECMA) -// KVChecksum is the checksum of a collection of key-value pairs. +// KVChecksum is the checksum of a collection of key-value pairs. The zero value +// of KVChecksum is a checksum for empty content and zero keyspace. type KVChecksum struct { base uint64 prefixLen int @@ -34,11 +35,9 @@ type KVChecksum struct { checksum uint64 } -// NewKVChecksum creates a new KVChecksum with the given checksum. -func NewKVChecksum(checksum uint64) *KVChecksum { - return &KVChecksum{ - checksum: checksum, - } +// NewKVChecksum creates a pointer to zero KVChecksum. +func NewKVChecksum() *KVChecksum { + return &KVChecksum{} } // NewKVChecksumWithKeyspace creates a new KVChecksum with the given checksum and keyspace. @@ -127,3 +126,135 @@ func (c *KVChecksum) MarshalJSON() ([]byte, error) { result := fmt.Sprintf(`{"checksum":%d,"size":%d,"kvs":%d}`, c.checksum, c.bytes, c.kvs) return []byte(result), nil } + +// KVGroupChecksum is KVChecksum(s) each for a data KV group or index KV groups. +type KVGroupChecksum struct { + m map[int64]*KVChecksum + codec tikv.Codec +} + +// DataKVGroupID represents the ID for data KV group, as index id starts from 1, +// so we use -1 to represent data kv group. +const DataKVGroupID = -1 + +// NewKVGroupChecksumWithKeyspace creates a new KVGroupChecksum with the given +// keyspace. +func NewKVGroupChecksumWithKeyspace(k tikv.Codec) *KVGroupChecksum { + m := make(map[int64]*KVChecksum, 8) + m[DataKVGroupID] = NewKVChecksumWithKeyspace(k) + return &KVGroupChecksum{m: m, codec: k} +} + +// NewKVGroupChecksumForAdd creates a new KVGroupChecksum, and it can't be used +// with UpdateOneDataKV or UpdateOneIndexKV. +func NewKVGroupChecksumForAdd() *KVGroupChecksum { + m := make(map[int64]*KVChecksum, 8) + m[DataKVGroupID] = NewKVChecksum() + return &KVGroupChecksum{m: m} +} + +// UpdateOneDataKV updates the checksum with a single data(record) key-value +// pair. It will not check the key-value pair's key is a real data key again. +func (c *KVGroupChecksum) UpdateOneDataKV(kv common.KvPair) { + c.m[DataKVGroupID].UpdateOne(kv) +} + +// UpdateOneIndexKV updates the checksum with a single index key-value pair. It +// will not check the key-value pair's key is a real index key of that index ID +// again. +func (c *KVGroupChecksum) UpdateOneIndexKV(indexID int64, kv common.KvPair) { + cksum := c.m[indexID] + if cksum == nil { + cksum = NewKVChecksumWithKeyspace(c.codec) + c.m[indexID] = cksum + } + cksum.UpdateOne(kv) +} + +// Add adds the checksum of another KVGroupChecksum. +func (c *KVGroupChecksum) Add(other *KVGroupChecksum) { + for id, cksum := range other.m { + thisCksum := c.getOrCreateOneGroup(id) + thisCksum.Add(cksum) + } +} + +func (c *KVGroupChecksum) getOrCreateOneGroup(id int64) *KVChecksum { + cksum, ok := c.m[id] + if ok { + return cksum + } + if c.codec == nil { + cksum = NewKVChecksum() + } else { + cksum = NewKVChecksumWithKeyspace(c.codec) + } + c.m[id] = cksum + return cksum +} + +// AddRawGroup adds the raw information of a KV group. +func (c *KVGroupChecksum) AddRawGroup(id int64, bytes, kvs, checksum uint64) { + oneGroup := c.getOrCreateOneGroup(id) + tmp := MakeKVChecksum(bytes, kvs, checksum) + oneGroup.Add(&tmp) +} + +// DataAndIndexSumSize returns the total size of data KV pairs and index KV pairs. +func (c *KVGroupChecksum) DataAndIndexSumSize() (dataSize, indexSize uint64) { + for id, cksum := range c.m { + if id == DataKVGroupID { + dataSize = cksum.SumSize() + } else { + indexSize += cksum.SumSize() + } + } + return +} + +// DataAndIndexSumKVS returns the total number of data KV pairs and index KV pairs. +func (c *KVGroupChecksum) DataAndIndexSumKVS() (dataKVS, indexKVS uint64) { + for id, cksum := range c.m { + if id == DataKVGroupID { + dataKVS = cksum.SumKVS() + } else { + indexKVS += cksum.SumKVS() + } + } + return +} + +// GetInnerChecksums returns a cloned map of index ID to its KVChecksum. +func (c *KVGroupChecksum) GetInnerChecksums() map[int64]*KVChecksum { + m := make(map[int64]*KVChecksum, len(c.m)) + for id, cksum := range c.m { + m[id] = &KVChecksum{ + base: cksum.base, + prefixLen: cksum.prefixLen, + bytes: cksum.bytes, + kvs: cksum.kvs, + checksum: cksum.checksum, + } + } + return m +} + +// MergedChecksum merges all groups of this checksum into a single KVChecksum. +func (c *KVGroupChecksum) MergedChecksum() KVChecksum { + merged := NewKVChecksum() + for _, cksum := range c.m { + merged.Add(cksum) + } + return *merged +} + +// MarshalLogObject implements the zapcore.ObjectMarshaler interface. +func (c *KVGroupChecksum) MarshalLogObject(encoder zapcore.ObjectEncoder) error { + for id, cksum := range c.m { + err := encoder.AddObject(fmt.Sprintf("id=%d", id), cksum) + if err != nil { + return err + } + } + return nil +} diff --git a/br/pkg/lightning/verification/checksum_test.go b/br/pkg/lightning/verification/checksum_test.go index 58484eecfea16..68adbf025cef6 100644 --- a/br/pkg/lightning/verification/checksum_test.go +++ b/br/pkg/lightning/verification/checksum_test.go @@ -21,12 +21,11 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/verification" "github.com/stretchr/testify/require" + "github.com/tikv/client-go/v2/tikv" ) -func uint64NotEqual(a uint64, b uint64) bool { return a != b } - -func TestChcksum(t *testing.T) { - checksum := verification.NewKVChecksum(0) +func TestChecksum(t *testing.T) { + checksum := verification.NewKVChecksum() require.Equal(t, uint64(0), checksum.Sum()) // checksum on nothing @@ -64,7 +63,7 @@ func TestChcksum(t *testing.T) { checksum.Update(kvs) require.Equal(t, kvBytes<<1, checksum.SumSize()) require.Equal(t, uint64(len(kvs))<<1, checksum.SumKVS()) - require.True(t, uint64NotEqual(checksum.Sum(), excpectChecksum)) + require.NotEqual(t, excpectChecksum, checksum.Sum()) } func TestChecksumJSON(t *testing.T) { @@ -79,3 +78,55 @@ func TestChecksumJSON(t *testing.T) { require.NoError(t, err) require.Equal(t, []byte(`{"Checksum":{"checksum":7890,"size":123,"kvs":456}}`), res) } + +type mockCodec struct { + tikv.Codec + keyspace []byte +} + +func (m *mockCodec) GetKeyspace() []byte { + return m.keyspace +} + +func TestGroupChecksum(t *testing.T) { + codec := &mockCodec{} + kvPair := common.KvPair{Key: []byte("key"), Val: []byte("val")} + kvPair2 := common.KvPair{Key: []byte("key2"), Val: []byte("val2")} + + c := verification.NewKVGroupChecksumWithKeyspace(codec) + c.UpdateOneDataKV(kvPair) + c.UpdateOneIndexKV(1, kvPair2) + inner := c.GetInnerChecksums() + require.Equal(t, 2, len(inner)) + require.Equal(t, uint64(1), inner[1].SumKVS()) + require.Equal(t, uint64(1), inner[verification.DataKVGroupID].SumKVS()) + + keyspaceCodec := &mockCodec{keyspace: []byte("keyspace")} + keyspaceC := verification.NewKVGroupChecksumWithKeyspace(keyspaceCodec) + keyspaceC.UpdateOneDataKV(kvPair) + keyspaceC.UpdateOneIndexKV(1, kvPair2) + keyspaceInner := keyspaceC.GetInnerChecksums() + require.NotEqual(t, inner, keyspaceInner) + + c2 := verification.NewKVGroupChecksumWithKeyspace(codec) + c2.UpdateOneIndexKV(1, kvPair) + c2.UpdateOneIndexKV(2, kvPair2) + c.Add(c2) + inner = c.GetInnerChecksums() + require.Equal(t, 3, len(inner)) + require.Equal(t, uint64(2), inner[1].SumKVS()) + require.Equal(t, uint64(1), inner[2].SumKVS()) + require.Equal(t, uint64(1), inner[verification.DataKVGroupID].SumKVS()) + + dataKVCnt, indexKVCnt := c.DataAndIndexSumKVS() + require.Equal(t, uint64(1), dataKVCnt) + require.Equal(t, uint64(3), indexKVCnt) + + dataSize, indexSize := c.DataAndIndexSumSize() + require.Equal(t, uint64(6), dataSize) + require.Equal(t, uint64(22), indexSize) + + merged := c.MergedChecksum() + require.Equal(t, uint64(4), merged.SumKVS()) + require.Equal(t, uint64(28), merged.SumSize()) +} diff --git a/br/tests/lightning_config_max_error/err_config.toml b/br/tests/lightning_config_max_error/err_config.toml index 0d135493244ce..4467bc582429f 100644 --- a/br/tests/lightning_config_max_error/err_config.toml +++ b/br/tests/lightning_config_max_error/err_config.toml @@ -5,4 +5,4 @@ threshold = 4 header = true [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_config_max_error/normal_config.toml b/br/tests/lightning_config_max_error/normal_config.toml index d6d4e55a4b8ec..1092efdf0c733 100644 --- a/br/tests/lightning_config_max_error/normal_config.toml +++ b/br/tests/lightning_config_max_error/normal_config.toml @@ -5,4 +5,4 @@ threshold = 20 header = true [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_config_max_error/normal_config_old_style.toml b/br/tests/lightning_config_max_error/normal_config_old_style.toml index fe402d071f5e0..c3408f00a2144 100644 --- a/br/tests/lightning_config_max_error/normal_config_old_style.toml +++ b/br/tests/lightning_config_max_error/normal_config_old_style.toml @@ -5,4 +5,4 @@ max-error = 0 # this actually sets the type error header = true [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_config_max_error/run.sh b/br/tests/lightning_config_max_error/run.sh index 74de7022007ce..897dda57a95df 100755 --- a/br/tests/lightning_config_max_error/run.sh +++ b/br/tests/lightning_config_max_error/run.sh @@ -25,6 +25,7 @@ data_file="${mydir}/data/mytest.testtbl.csv" total_row_count=$( sed '1d' "${data_file}" | wc -l | xargs echo ) uniq_row_count=$( sed '1d' "${data_file}" | awk -F, '{print $1}' | sort | uniq -c | awk '{print $1}' | grep -c '1' | xargs echo ) duplicated_row_count=$(( ${total_row_count} - ${uniq_row_count} )) +remaining_row_count=$(( ${uniq_row_count} + ${duplicated_row_count}/2 )) run_sql 'DROP TABLE IF EXISTS mytest.testtbl' run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' @@ -64,7 +65,7 @@ check_contains "COUNT(*): ${duplicated_row_count}" # Check remaining records in the target table run_sql 'SELECT COUNT(*) FROM mytest.testtbl' -check_contains "COUNT(*): ${uniq_row_count}" +check_contains "COUNT(*): ${remaining_row_count}" # import a third time @@ -78,13 +79,13 @@ check_contains "COUNT(*): ${duplicated_row_count}" # Check remaining records in the target table run_sql 'SELECT COUNT(*) FROM mytest.testtbl' -check_contains "COUNT(*): ${uniq_row_count}" +check_contains "COUNT(*): ${remaining_row_count}" # Check tidb backend record duplicate entry in conflict_records table run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_records' run_lightning --backend tidb --config "${mydir}/tidb.toml" run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_records' -check_contains "COUNT(*): 10" +check_contains "COUNT(*): 15" run_sql 'SELECT * FROM lightning_task_info.conflict_records WHERE offset = 149' check_contains "error: Error 1062 (23000): Duplicate entry '5' for key 'testtbl.PRIMARY'" check_contains "row_data: ('5','bbb05')" diff --git a/br/tests/lightning_config_skip_csv_header/err_config.toml b/br/tests/lightning_config_skip_csv_header/err_config.toml index 95493db0dff44..0fb86faf9643c 100644 --- a/br/tests/lightning_config_skip_csv_header/err_config.toml +++ b/br/tests/lightning_config_skip_csv_header/err_config.toml @@ -6,4 +6,4 @@ header = true header-schema-match = true [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_config_skip_csv_header/err_default_config.toml b/br/tests/lightning_config_skip_csv_header/err_default_config.toml index a7b17c7276d92..6df483fb9e152 100644 --- a/br/tests/lightning_config_skip_csv_header/err_default_config.toml +++ b/br/tests/lightning_config_skip_csv_header/err_default_config.toml @@ -5,4 +5,4 @@ check-requirements=true header = true [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_config_skip_csv_header/normal_config.toml b/br/tests/lightning_config_skip_csv_header/normal_config.toml index 190e635cfc4e9..4572ebe0b2fd5 100644 --- a/br/tests/lightning_config_skip_csv_header/normal_config.toml +++ b/br/tests/lightning_config_skip_csv_header/normal_config.toml @@ -6,4 +6,4 @@ header = true header-schema-match = false [tikv-importer] -duplicate-resolution = 'remove' +duplicate-resolution = 'replace' diff --git a/br/tests/lightning_duplicate_detection/config1.toml b/br/tests/lightning_duplicate_detection/config1.toml index 6497e9e30949b..09f3ed0528ea9 100644 --- a/br/tests/lightning_duplicate_detection/config1.toml +++ b/br/tests/lightning_duplicate_detection/config1.toml @@ -5,7 +5,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" -duplicate-resolution = 'record' +duplicate-resolution = 'replace' incremental-import = true [checkpoint] diff --git a/br/tests/lightning_duplicate_detection/config2.toml b/br/tests/lightning_duplicate_detection/config2.toml index 760f50168508a..17e840fd45250 100644 --- a/br/tests/lightning_duplicate_detection/config2.toml +++ b/br/tests/lightning_duplicate_detection/config2.toml @@ -5,7 +5,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" -duplicate-resolution = 'record' +duplicate-resolution = 'replace' incremental-import = true [checkpoint] diff --git a/br/tests/lightning_duplicate_resolution_remove/config.toml b/br/tests/lightning_duplicate_resolution_error/config.toml similarity index 89% rename from br/tests/lightning_duplicate_resolution_remove/config.toml rename to br/tests/lightning_duplicate_resolution_error/config.toml index 4075e7d791993..b64c2df3e8618 100644 --- a/br/tests/lightning_duplicate_resolution_remove/config.toml +++ b/br/tests/lightning_duplicate_resolution_error/config.toml @@ -3,7 +3,7 @@ task-info-schema-name = 'lightning_task_info' [tikv-importer] backend = 'local' -duplicate-resolution = 'remove' +duplicate-resolution = 'error' add-index-by-sql = false [checkpoint] diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_error/data/dup_resolve-schema-create.sql similarity index 100% rename from br/tests/lightning_duplicate_resolution_remove/data/dup_resolve-schema-create.sql rename to br/tests/lightning_duplicate_resolution_error/data/dup_resolve-schema-create.sql diff --git a/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..113efc3929c48 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key, + b int not null, + c text, + key key_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..099a240a110ee --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error/data/dup_resolve.a.1.csv @@ -0,0 +1,6 @@ +a,b,c +1,6,1.csv +2,6,2.csv +3,3,3.csv +3,4,4.csv +5,4,5.csv diff --git a/br/tests/lightning_duplicate_resolution_error/run.sh b/br/tests/lightning_duplicate_resolution_error/run.sh new file mode 100644 index 0000000000000..6584eb48ee70a --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error/run.sh @@ -0,0 +1,31 @@ +#!/bin/bash +# +# 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, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eux + +check_cluster_version 5 2 0 'duplicate detection' || exit 0 + +mydir=$(dirname "${BASH_SOURCE[0]}") + +run_sql 'DROP TABLE IF EXISTS dup_resolve.a' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' + +! run_lightning --backend local --config "${mydir}/config.toml" +[ $? -eq 0 ] + +tail -n 10 $TEST_DIR/lightning.log | grep "ERROR" | tail -n 1 | grep -Fq "[Lightning:Restore:ErrFoundDuplicateKey]found duplicate key" + +check_not_contains "the whole procedure completed" $TEST_DIR/lightning.log diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/config.toml b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/config.toml new file mode 100644 index 0000000000000..41ab032d9d6fa --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'error' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 2 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..113efc3929c48 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key, + b int not null, + c text, + key key_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..c46afaa19de4e --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.1.csv @@ -0,0 +1,6 @@ +a,b,c +1,1,1.csv +2,2,2.csv +3,3,3.csv +4,4,4.csv +5,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.2.csv b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.2.csv new file mode 100644 index 0000000000000..66df880540662 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/data/dup_resolve.a.2.csv @@ -0,0 +1,5 @@ +a,b,c +1,10,10.csv +7,7,7.csv +8,8,8.csv +9,9,9.csv diff --git a/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/run.sh b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/run.sh new file mode 100644 index 0000000000000..6584eb48ee70a --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_pk_multiple_files/run.sh @@ -0,0 +1,31 @@ +#!/bin/bash +# +# 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, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eux + +check_cluster_version 5 2 0 'duplicate detection' || exit 0 + +mydir=$(dirname "${BASH_SOURCE[0]}") + +run_sql 'DROP TABLE IF EXISTS dup_resolve.a' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' + +! run_lightning --backend local --config "${mydir}/config.toml" +[ $? -eq 0 ] + +tail -n 10 $TEST_DIR/lightning.log | grep "ERROR" | tail -n 1 | grep -Fq "[Lightning:Restore:ErrFoundDuplicateKey]found duplicate key" + +check_not_contains "the whole procedure completed" $TEST_DIR/lightning.log diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/config.toml b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/config.toml new file mode 100644 index 0000000000000..41ab032d9d6fa --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'error' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 2 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..9abf38da045fd --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key, + b int not null, + c text, + unique key key_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..c46afaa19de4e --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.1.csv @@ -0,0 +1,6 @@ +a,b,c +1,1,1.csv +2,2,2.csv +3,3,3.csv +4,4,4.csv +5,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.2.csv b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.2.csv new file mode 100644 index 0000000000000..16fa5cf896a6a --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/data/dup_resolve.a.2.csv @@ -0,0 +1,5 @@ +a,b,c +6,1,1.csv +7,7,7.csv +8,8,8.csv +9,9,9.csv diff --git a/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/run.sh b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/run.sh new file mode 100644 index 0000000000000..6584eb48ee70a --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_error_uk_multiple_files/run.sh @@ -0,0 +1,31 @@ +#!/bin/bash +# +# 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, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +set -eux + +check_cluster_version 5 2 0 'duplicate detection' || exit 0 + +mydir=$(dirname "${BASH_SOURCE[0]}") + +run_sql 'DROP TABLE IF EXISTS dup_resolve.a' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' + +! run_lightning --backend local --config "${mydir}/config.toml" +[ $? -eq 0 ] + +tail -n 10 $TEST_DIR/lightning.log | grep "ERROR" | tail -n 1 | grep -Fq "[Lightning:Restore:ErrFoundDuplicateKey]found duplicate key" + +check_not_contains "the whole procedure completed" $TEST_DIR/lightning.log diff --git a/br/tests/lightning_duplicate_resolution_incremental/config1.toml b/br/tests/lightning_duplicate_resolution_incremental/config1.toml index 4ecb29972d56a..ea6fdd300f320 100644 --- a/br/tests/lightning_duplicate_resolution_incremental/config1.toml +++ b/br/tests/lightning_duplicate_resolution_incremental/config1.toml @@ -5,7 +5,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" -duplicate-resolution = "remove" +duplicate-resolution = "replace" incremental-import = true [checkpoint] diff --git a/br/tests/lightning_duplicate_resolution_incremental/config2.toml b/br/tests/lightning_duplicate_resolution_incremental/config2.toml index c4af2d6e93a9d..39969a670b4f5 100644 --- a/br/tests/lightning_duplicate_resolution_incremental/config2.toml +++ b/br/tests/lightning_duplicate_resolution_incremental/config2.toml @@ -5,7 +5,7 @@ table-concurrency = 10 [tikv-importer] backend = "local" -duplicate-resolution = "remove" +duplicate-resolution = "replace" incremental-import = true [checkpoint] diff --git a/br/tests/lightning_duplicate_resolution_incremental/run.sh b/br/tests/lightning_duplicate_resolution_incremental/run.sh index 1f8f751eb477e..50dc4f5b21b8e 100644 --- a/br/tests/lightning_duplicate_resolution_incremental/run.sh +++ b/br/tests/lightning_duplicate_resolution_incremental/run.sh @@ -56,8 +56,8 @@ run_sql 'admin check table dup_resolve_detect.ta' # Check data correctness run_sql 'select count(*), sum(id) from dup_resolve_detect.ta where id < 100' -check_contains 'count(*): 10' -check_contains 'sum(id): 80' +check_contains 'count(*): 15' +check_contains 'sum(id): 120' run_sql 'select count(*), sum(id) from dup_resolve_detect.ta where id > 100' check_contains 'count(*): 16' diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a-schema.sql deleted file mode 100644 index 897f9f20e1aae..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a-schema.sql +++ /dev/null @@ -1,7 +0,0 @@ -create table a ( - a int primary key clustered, - b int not null unique, - c text, - d int generated always as (a + b), - key (d) -); diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.1.csv deleted file mode 100644 index 3495e40d701e3..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.1.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,4,1.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.2.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.2.csv deleted file mode 100644 index de89d220d0af1..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.2.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,5,2.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.3.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.3.csv deleted file mode 100644 index 08370bbb2e4ca..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.3.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,6,3.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.4.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.4.csv deleted file mode 100644 index a22ce6d2bd239..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.4.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -2,1,4.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.5.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.5.csv deleted file mode 100644 index f4f1abd3669ea..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.5.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -3,30,5.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.6.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.6.csv deleted file mode 100644 index 3c4367dbc5d2e..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.6.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -3,30,6.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.7.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.7.csv deleted file mode 100644 index 7631a73641966..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.7.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -4,2,7.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.8.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.8.csv deleted file mode 100644 index 4fcd577f80475..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.8.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -5,2,8.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.9.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.9.csv deleted file mode 100644 index 740072446a27d..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.a.9.csv +++ /dev/null @@ -1,4 +0,0 @@ -a,b,c -5,7,9.csv#1 -6,8,9.csv#2 -6,9,9.csv#3 diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b-schema.sql b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b-schema.sql deleted file mode 100644 index 925fc25f1e8a9..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b-schema.sql +++ /dev/null @@ -1,5 +0,0 @@ -create table a ( - a int primary key nonclustered, - b int unique, - c varchar(100) unique -); diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.1.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.1.csv deleted file mode 100644 index 3495e40d701e3..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.1.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,4,1.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.2.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.2.csv deleted file mode 100644 index de89d220d0af1..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.2.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,5,2.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.3.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.3.csv deleted file mode 100644 index 08370bbb2e4ca..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.3.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -1,6,3.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.4.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.4.csv deleted file mode 100644 index a22ce6d2bd239..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.4.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -2,1,4.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.5.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.5.csv deleted file mode 100644 index f4f1abd3669ea..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.5.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -3,30,5.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.6.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.6.csv deleted file mode 100644 index 3c4367dbc5d2e..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.6.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -3,30,6.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.7.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.7.csv deleted file mode 100644 index 7631a73641966..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.7.csv +++ /dev/null @@ -1,2 +0,0 @@ -a,b,c -4,2,7.csv diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.8.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.8.csv deleted file mode 100644 index 7f8b0d9ccdc2f..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.8.csv +++ /dev/null @@ -1,4 +0,0 @@ -a,b,c -5,2,8.csv#1 -7,\N,8.csv#2 -8,\N,8.csv#3 diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.9.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.9.csv deleted file mode 100644 index 740072446a27d..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.b.9.csv +++ /dev/null @@ -1,4 +0,0 @@ -a,b,c -5,7,9.csv#1 -6,8,9.csv#2 -6,9,9.csv#3 diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c-schema.sql b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c-schema.sql deleted file mode 100644 index af55f189bf776..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c-schema.sql +++ /dev/null @@ -1,7 +0,0 @@ -create table c ( - a double not null, - b decimal not null, - c text, - primary key (a, b) clustered, - unique key (b, c(10)) -); diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.1.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.1.csv deleted file mode 100644 index bf68021fda093..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.1.csv +++ /dev/null @@ -1,9 +0,0 @@ -a,b,c -2,1,1.csv#1 -3,0,1.csv#2 -5,1,1.csv#3 -7,0,1.csv#4 -1,1,1.csv#5 -3,0,1.csv#6 -7,1,1.csv#7 -9,0,1.csv#8 diff --git a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.2.csv b/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.2.csv deleted file mode 100644 index 2486d72cfc90f..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/data/dup_resolve.c.2.csv +++ /dev/null @@ -1,9 +0,0 @@ -a,b,c -1,1,2.csv#1 -4,0,2.csv#2 -9,1,2.csv#3 -6,0,2.csv#4 -5,1,2.csv#5 -6,0,2.csv#6 -9,1,2.csv#7 -4,0,2.csv#8 diff --git a/br/tests/lightning_duplicate_resolution_remove/run.sh b/br/tests/lightning_duplicate_resolution_remove/run.sh deleted file mode 100644 index f98a4c3ee8e2b..0000000000000 --- a/br/tests/lightning_duplicate_resolution_remove/run.sh +++ /dev/null @@ -1,73 +0,0 @@ -#!/bin/bash -# -# 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, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -set -eux - -check_cluster_version 5 2 0 'duplicate detection' || exit 0 - -for i in {1..2} ; do - run_lightning - - # Ensure all tables are consistent. - run_sql 'admin check table dup_resolve.a' - run_sql 'admin check table dup_resolve.b' - run_sql 'admin check table dup_resolve.c' - - ## Table "a" has a clustered integer key and generated column. - - # only one row remains (2, 1, 4.csv). all others are duplicates 🤷 - run_sql 'select count(*) from dup_resolve.a' - check_contains 'count(*): 1' - - run_sql 'select * from dup_resolve.a' - check_contains 'a: 2' - check_contains 'b: 1' - check_contains 'c: 4.csv' - check_contains 'd: 3' - - ## Table "b" has a nonclustered integer key and nullable unique column. - run_sql 'select count(*) from dup_resolve.b' - check_contains 'count(*): 3' - - run_sql 'select * from dup_resolve.b where a = 2' - check_contains 'b: 1' - check_contains 'c: 4.csv' - run_sql 'select * from dup_resolve.b where a = 7' - check_contains 'b: NULL' - check_contains 'c: 8.csv#2' - run_sql 'select * from dup_resolve.b where a = 8' - check_contains 'b: NULL' - check_contains 'c: 8.csv#3' - - ## Table "c" has a clustered non-integer key. - run_sql 'select count(*) from dup_resolve.c' - check_contains 'count(*): 4' - - run_sql 'select c from dup_resolve.c where a = 2 and b = 1' - check_contains 'c: 1.csv#1' - run_sql 'select c from dup_resolve.c where a = 7 and b = 0' - check_contains 'c: 1.csv#4' - run_sql 'select c from dup_resolve.c where a = 7 and b = 1' - check_contains 'c: 1.csv#7' - run_sql 'select c from dup_resolve.c where a = 9 and b = 0' - check_contains 'c: 1.csv#8' - - # test can use DELETE to cleanup data - run_sql 'delete from dup_resolve.a' - run_sql 'truncate table dup_resolve.b' # TODO: non-clustered index table should also support DELETE - run_sql 'delete from dup_resolve.c' - run_sql 'drop database lightning_task_info' -done diff --git a/br/tests/lightning_issue_40657/config.toml b/br/tests/lightning_issue_40657/config.toml index 7ffe00f44c1fe..e5d268f5cd3ab 100644 --- a/br/tests/lightning_issue_40657/config.toml +++ b/br/tests/lightning_issue_40657/config.toml @@ -1,6 +1,6 @@ [tikv-importer] backend = "local" -duplicate-resolution = "remove" +duplicate-resolution = "replace" add-index-by-sql = false [mydumper.csv] diff --git a/br/tests/lightning_issue_40657/run.sh b/br/tests/lightning_issue_40657/run.sh index a5ab8e88298cc..4d2934a5ca24e 100644 --- a/br/tests/lightning_issue_40657/run.sh +++ b/br/tests/lightning_issue_40657/run.sh @@ -23,7 +23,7 @@ check_cluster_version 5 2 0 'duplicate detection' || exit 0 run_lightning -d "$CUR/data1" run_sql 'admin check table test.t' run_sql 'select count(*) from test.t' -check_contains 'count(*): 3' +check_contains 'count(*): 4' run_sql 'select count(*) from lightning_task_info.conflict_error_v2' check_contains 'count(*): 2' diff --git a/br/tests/run_group_lightning_tests.sh b/br/tests/run_group_lightning_tests.sh index e84cb75a5317e..abee364fe2d74 100755 --- a/br/tests/run_group_lightning_tests.sh +++ b/br/tests/run_group_lightning_tests.sh @@ -22,8 +22,8 @@ declare -A groups groups=( ["G00"]='lightning_auto_random_default lightning_bom_file lightning_character_sets lightning_check_partial_imported lightning_checkpoint lightning_checkpoint_chunks lightning_checkpoint_columns lightning_checkpoint_dirty_tableid' ["G01"]='lightning_checkpoint_engines lightning_checkpoint_engines_order lightning_checkpoint_error_destroy lightning_checkpoint_parquet lightning_checkpoint_timestamp lightning_checksum_mismatch lightning_cmdline_override lightning_column_permutation lightning_common_handle lightning_compress lightning_concurrent-restore' - ["G02"]='lightning_config_max_error lightning_config_skip_csv_header lightning_csv lightning_default-columns lightning_disable_scheduler_by_key_range lightning_disk_quota lightning_distributed_import lightning_drop_other_tables_halfway lightning_duplicate_detection lightning_duplicate_detection_new lightning_duplicate_resolution_incremental lightning_duplicate_resolution_remove lightning_duplicate_resolution_replace_multiple_keys_clustered_pk' - ["G03"]='lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk lightning_duplicate_resolution_replace_one_key lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk' + ["G02"]='lightning_config_max_error lightning_config_skip_csv_header lightning_csv lightning_default-columns lightning_disable_scheduler_by_key_range lightning_disk_quota lightning_distributed_import lightning_drop_other_tables_halfway lightning_duplicate_detection lightning_duplicate_detection_new lightning_duplicate_resolution_error lightning_duplicate_resolution_error_pk_multiple_files lightning_duplicate_resolution_error_uk_multiple_files lightning_duplicate_resolution_incremental' + ["G03"]='lightning_duplicate_resolution_replace_multiple_keys_clustered_pk lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk lightning_duplicate_resolution_replace_one_key lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk' ["G04"]='lightning_duplicate_resolution_replace_one_unique_key_clustered_pk lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk lightning_error_summary lightning_examples lightning_exotic_filenames lightning_extend_routes' ["G05"]='lightning_fail_fast lightning_fail_fast_on_nonretry_err lightning_file_routing lightning_foreign_key lightning_gcs lightning_generated_columns lightning_ignore_columns lightning_import_compress lightning_incremental lightning_issue_282 lightning_issue_40657 lightning_issue_410 lightning_issue_519 lightning_local_backend lightning_max_incr' ["G06"]='lightning_max_random lightning_multi_valued_index lightning_new_collation lightning_no_schema lightning_parquet lightning_partition_incremental lightning_partitioned-table lightning_record_network lightning_reload_cert lightning_restore lightning_routes lightning_routes_panic lightning_row-format-v2 lightning_s3' diff --git a/docs/design/2023-11-29-priority-queue-for-auto-analyze.md b/docs/design/2023-11-29-priority-queue-for-auto-analyze.md new file mode 100644 index 0000000000000..b863ba908cdd6 --- /dev/null +++ b/docs/design/2023-11-29-priority-queue-for-auto-analyze.md @@ -0,0 +1,367 @@ +# Priority Queue for Auto Analyze + +- Author(s): [Rustin Liu](http://github.com/hi-rustin) +- Discussion PR: +- Tracking Issue: + +## Table of Contents + +- [Priority Queue for Auto Analyze](#priority-queue-for-auto-analyze) + - [Table of Contents](#table-of-contents) + - [Introduction](#introduction) + - [Motivation or Background](#motivation-or-background) + - [Detailed Design](#detailed-design) + - [Partitioned Table](#partitioned-table) + - [Failed analysis](#failed-analysis) + - [Dataflow](#dataflow) + - [Frequently Asked Questions](#frequently-asked-questions) + - [Test Design](#test-design) + - [Functional Tests](#functional-tests) + - [Compatibility Tests](#compatibility-tests) + - [Performance Tests](#performance-tests) + - [Impacts \& Risks](#impacts--risks) + - [Investigation \& Alternatives](#investigation--alternatives) + - [CRDB](#crdb) + - [Basic Idea](#basic-idea) + - [Implementation Details](#implementation-details) + - [Frequently Asked Questions](#frequently-asked-questions-1) + - [MySQL](#mysql) + - [Basic Idea](#basic-idea-1) + - [Implementation Details](#implementation-details-1) + - [Frequently Asked Questions](#frequently-asked-questions-2) + - [SQL Server](#sql-server) + - [Basic Idea](#basic-idea-2) + - [Unresolved Questions](#unresolved-questions) + +## Introduction + +Auto analyze is a background job that automatically collects statistics for tables. This design proposes a priority queue for auto analyze to improve the efficiency of auto analyze. + +## Motivation or Background + +We have received numerous complaints in the past regarding the auto-analysis tasks. If users have many tables, we must assist them in analyzing the tables in the background to ensure up-to-date statistics are available for generating the best plan in the optimizer. +There are some complaints from users: + +1. The automatic analysis retry mechanism has flaws. + 1. If one table analysis keeps failing, it will halt the entire auto-analysis process. +2. Automatic analysis may encounter starvation issues. + 1. The analysis of small tables is being delayed due to the analysis of some large tables. +3. Due to the random selection algorithm, some tables cannot be analyzed for a long time. + +So we need to design a priority queue to solve these problems. + +## Detailed Design + +Before we design the priority queue, we need to know the current auto analyze process. + +1. During the TiDB bootstrap process, we initiate the load and update of the stats worker. [session.go?L3470:15] +2. We spawn a dedicated worker to perform automated analysis tasks. [domain.go?L2457:19] +3. For every stats lease, we will trigger an auto-analysis check: [domain.go?L2469:17] + > Please note that we only run it on the TiDB owner instance. +4. Check if we are currently within the allowed timeframe to perform the auto-analysis. [autoanalyze.go?L149:15] +5. To prevent the process from getting stuck on the same table's analysis failing, randomize the order of databases and tables. [autoanalyze.go?L155:5] +6. Try to execute the auto-analysis SQL. [autoanalyze.go?L243:6] + 1. **If the statistics are pseudo, it means they have not been loaded yet, so there is no need to analyze them.** [autoanalyze.go?L247:5] + 2. **If the table is too small, it may not be worth analyzing. We should focus on larger tables.** [autoanalyze.go?L247:5] + 3. **If the table has never been analyzed, we need to analyze it.** [autoanalyze.go?L287:6] + 4. **If the table reaches the autoAnalyzeRatio, we need to analyze it.** [autoanalyze.go?L287:6] + > We use `modify_count/count` to calculate the ratio. [autoanalyze.go?L301:40] + 5. **If the indexes have no statistics, we need to analyze them anyway.** [autoanalyze.go?L262:15] + +[session.go?L3470:15]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/session/session.go?L3470:15 +[domain.go?L2457:19]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/domain/domain.go?L2457:19&popover=pinned +[domain.go?L2469:17]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/domain/domain.go?L2469:17&popover=pinned +[autoanalyze.go?L149:15]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L149:15&popover=pinned +[autoanalyze.go?L155:5]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L155:5&popover=pinned +[autoanalyze.go?L243:6]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L243:6&popover=pinned +[autoanalyze.go?L247:5]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L247:5&popover=pinned +[autoanalyze.go?L287:6]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L287:6&popover=pinned +[autoanalyze.go?L301:40]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L301:40&popover=pinned +[autoanalyze.go?L262:15]: https://sourcegraph.com/github.com/hi-rustin/tidb@d4618d4a5f91ee3703336fd1ba328c2e477652e5/-/blob/pkg/statistics/handle/autoanalyze/autoanalyze.go?L262:15&popover=pinned + +The above process is the current auto analyze process. We can see that the current auto analyze process is a random selection algorithm. We need to design a priority queue to solve the above problems. + +Since we perform analysis tasks synchronously on a single node for each table, we need to carry out weighted sorting on the tables that need analysis. + +1. Only the total count of the table more than 1000 rows will be considered. +2. We still use tidb_auto_analyze_ratio to determine if the table needs to be analyzed. The default value is 0.5. It means that auto-analyze is triggered when greater than 50% of the rows in a table have been modified. +3. If a table is added to the auto-analysis queue, a weight must be assigned to it to determine the execution order. + +The refresh time of the auto-analyze queue is not only determined by the frequency we set, but also depends on the execution time of the previous analysis task. Therefore, we can safely continue to use the old scheme of refreshing the queue every 3 seconds. This will only lead to excessive CPU consumption for auto-analyze checks when the entire cluster is idle, which is acceptable for us at the moment. In the future, we can completely solve this issue by updating the queue instead of rebuilding the entire queue. + +Weight table: +| **Name** | **Meaning** | **Weight** | +|----------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| Percentage of Change | The percentage of change since the last analysis. Note: For unanalyzed tables, we set the percentage of changes to 100%. | log10(1 + Change Ratio). [Check the graph](https://www.desmos.com/calculator/gb3i2cropz). **Note: For unanalyzed tables, we set the percentage of changes to 100%** | +| Table Size | The size is equal to the number of rows multiplied by the number of columns in the table that are subject to analysis. The smaller tables should have a higher priority than the bigger ones. | Applying a logarithmic transformation, namely log10(1 + Table Size), and its 'penalty' is calculated as 1 - log10(1 + Table Size). [Check the graph.](https://www.desmos.com/calculator/x3fq76w8sb) | +| Analysis Interval | Time since the last analysis execution for the table. The bigger interval should have a higher priority than the smaller interval. | Applying a logarithmic transformation, namely log10(1 + Analysis Interval). To further compress the rate of growth for larger values, we can consider taking the logarithmic square root of x'. The final formula is log10(1 + √Analysis Interval). [Check the graph.](https://www.desmos.com/calculator/plhtkfqhx9) | +| Special Event | For example, the table has a new index but it hasn't been analyzed yet. | HasNewindexWithoutStats: 2 | + +We need to design a formula that ensures the three variables (Change Ratio of the table, Size of the table, and the Time Interval since the last analysis) maintain specific proportions when calculating weights. Here is a basic idea: + +- Table Change Ratio (Change Ratio): Accounts for 60% +- Table Size (Size): Accounts for 10% +- Analysis Interval (Analysis Interval): Accounts for 30% + +The calculation formula is: priority_score = ($$0.6 \times \log_{10}(1 + \text{Change Ratio}) + 0.1 \times (1 - \log_{10}(1 + \text{Table Size})) + 0.3 \times \log_{10}(1 + \sqrt{\text{Analysis Interval}})$$ + special_event[event]) +The calculation formula is: `priority_score = (change_percentage[size] * last_failed_time_weight[interval] * special_event[event])` + +The ratio mentioned above is determined based on our current sample data. We need more tests to ascertain a more accurate ratio. Furthermore, we will expose these ratios as some configurations, as more precise control and adjustments may be necessary in different scenarios. + +### Partitioned Table + +For partitioned tables, if the pruning mode is static, then we don't need to merge the global statistics, so we can consider it as a normal table and calculate the weight for it. +But if the pruning mode is dynamic, then we need to get all the partitions that need to be analyzed and calculate the average percentage of changes, and consider it as a single item in the priority queue. + +Pseudocode: + +```go +function calculateAvgChangeForPartitions(partitionStats, defs, autoAnalyzeRatio): + totalChangePercent = 0 + count = 0 + partitionNames = [] + + for each def in defs: + tblStats = partitionStats[def.ID] + changePercent = calculateChangePercentage(tblStats, autoAnalyzeRatio) + if changePercent is 0: + continue + + totalChangePercent += changePercent + append def.Name.O to partitionNames + count += 1 + + avgChange = totalChangePercent / count + return avgChange, partitionNames + +function calculateChangePercentage(tblStats, autoAnalyzeRatio): + if tblStats.Pseudo or tblStats.RealtimeCount < AutoAnalyzeMinCnt: + return 0 + + if not TableAnalyzed(tblStats): + return 1 + + tblCnt = tblStats.RealtimeCount + if histCnt = tblStats.GetAnalyzeRowCount() > 0: + tblCnt = histCnt + + res = tblStats.ModifyCount / tblCnt + if res > autoAnalyzeRatio: + return res + + return 0 +``` + +### Failed analysis + +Sometimes we may encounter some problem when we analyze a table, we need to avoid analyzing the same table again and again. So after we select a table from the priority queue. We need to make sure that it is valid to be analyzed. +We check the interval between now and the last failed analysis ends time to determine if we need to analyze the selected table. + +**The calculation rule is: if interval >= 2 * average automatic analysis interval then we thought it was a valid table to be analyzed. We only compute it after we get it from the priority queue, which would help us save a lot of resources. Because getting this information from TiKV is very expensive.** + +Pseudocode: + +```go +function IsValidToAnalyze(j): + if j.Weight is 0: + return false + + lastFailedAnalysisDuration, err1 = getLastFailedAnalysisDuration(j.DBName, j.TableName, "") + if err1 is not nil: + return false + + averageAnalysisDuration, err2 = getAverageAnalysisDuration(j.DBName, j.TableName, "") + if err2 is not nil: + return false + + // Failed analysis duration is less than 2 times the average analysis duration. + // Skip this table to avoid too many failed analysis. + if lastFailedAnalysisDuration < 2 * averageAnalysisDuration: + return false + + return true +``` + +### Dataflow + +Pick One Table From The Priority Queue (default: every 3s) + +![Dataflow](./imgs/analyze-dataflow.png) + +### Frequently Asked Questions + +1. How many auto-analysis tasks can we run at the same time in the cluster? + + Only one. We only execute the auto-analysis background worker and task on the owner node. + +2. What happens if we analyze the same table multiple times at the same time? + + It will use the most recent successful analysis result. + +3. Why don't we simply separate the queues for large tables and small tables? + + Because currently in our entire cluster, only the owner node can submit tasks on its own instance. Even if we divide into two queues, we will still encounter situations of mutual blocking. + Unless we can submit tasks for both large and small tables simultaneously on the owner node. But I'm afraid this would put additional pressure on that node. + Additionally, we cannot simply determine what constitutes a large table and what constitutes a small table. Therefore, weighting based on the number of rows might be more reasonable. + +4. How to get the last failed automatic analysis time? + + We can find the latest failed analysis from mysql.analyze_jobs. It has a fail_reason column. + +5. How do we identify if a table has a new index? + + During the bootstrap process, we load statistics for both indexes and columns and store them in a cache. This cache allows us to identify if a table has a new index without statistics. + +6. How do we ascertain if a table has never been analyzed? + + We utilize the cache to verify if there are no loaded statistics for any indexes or columns of a table. If the cache doesn't contain any statistics for a table, it indicates that the table has never been analyzed. + +## Test Design + +This feature requires a focus on both correctness and performance tests. The primary objective of the correctness tests is to validate the accuracy of priority calculations. Performance tests aim to ensure that the priority queue's operation doesn't negatively impact system performance. + +### Functional Tests + +These tests should cover all potential scenarios involving the priority queue. For instance, the priority queue should correctly handle situations such as: + +1. A table that has never undergone analysis. +2. A table that has been analyzed, but its index lacks statistical data. +3. Multiple tables experiencing significant changes, requiring the priority queue to assign appropriate priorities. +4. A single table undergoing substantial changes, but analysis fails. In this case, the priority queue should assign the correct priority based on the last failed interval. +5. Mix of the above scenarios. + +### Compatibility Tests + +This feature is designed to seamlessly integrate with all existing functionalities, ensuring that the introduction of the priority queue does not compromise the accuracy of the auto-analyze process. + +To provide users with control, we will introduce a new system variable that allows the enabling or disabling of the priority queue. By default, this feature will be set to `OFF`. + +Following extensive testing and validation, we may consider setting the priority queue as the default option in future iterations. + +### Performance Tests + +Calculating the priority score for each table should not negatively impact the system's performance. We will perform extensive testing to ensure that the priority queue does not introduce any performance issues. + +## Impacts & Risks + +After the priority queue is enabled, the auto analyze process will be changed from random selection to weighted sorting. From the perspective of the user, the auto analyze process will be more reasonable. + +## Investigation & Alternatives + +### CRDB + +#### Basic Idea + +Statistics are refreshed in the following cases: + +- When there are no statistics. +- When it has been a long time since the last refresh, where "long time" is based on a moving average of the time across the last several refreshes. +- After a successful `IMPORT` or `RESTORE` into the table. +- After any schema change affecting the table. +- After each mutation operation (`INSERT`, `UPDATE`, or `DELETE`), the probability of a refresh is calculated using a formula that takes the cluster settings shown in the following table as inputs. These settings define the target number of rows in a table that must be stale before statistics on that table are refreshed. Increasing either setting will reduce the frequency of refreshes. In particular, `min_stale_rows` impacts the frequency of refreshes for small tables, while `fraction_stale_rows` has more of an impact on larger tables. + +| Setting | Default Value | Details | +|------------------------------------------------------|---------------|---------------------------------------------------------------------------------------| +| `sql.stats.automatic_collection.fraction_stale_rows` | 0.2 | Target fraction of stale rows per table that will trigger a statistics refresh. | +| `sql.stats.automatic_collection.min_stale_rows` | 500 | Target minimum number of stale rows per table that will trigger a statistics refresh. | + +You can configure automatic statistics collection on a per-table basis. + +#### Implementation Details + +1. Each server will create a refresher to refresh the stats periodically. [server_sql.go?L1126:2] +2. The refresher spawns a goroutine to try to trigger a refresh every minute. [automatic_stats.go?L438:11] +3. Refreshers use a map to store all mutation counts and use it as affected rows to try to trigger a refresh. [automatic_stats.go?L518:10] +4. `maybeRefreshStats` implements core logic. + 1. Use the average full refresh time to check if too much time has passed since the last refresh. [automatic_stats.go?L817:3] + 2. Use `statsFractionStaleRows` and `statsMinStaleRows` to calculate target rows: `targetRows := int64(rowCount*statsFractionStaleRows) + statsMinStaleRows` + 3. Generate a non-negative pseudo-random number in the half-open interval `[0,targetRows)` to check if it needs to trigger a refresh. [automatic_stats.go?L836:6] + 4. Try to refresh the table. This function will execute SQL through the CRDB job framework: `CREATE STATISTICS %s FROM [%d] WITH OPTIONS THROTTLING %% AS OF SYSTEM TIME '-%s'` [automatic_stats.go?L843:14] + 5. If it meets `ConcurrentCreateStatsError` + 1. If it must be refreshed, then set the `rowsAffected` to 0, so that we don't force a refresh if another node has already done it. + 2. If it is not a must-be-refreshed table, we ensure that the refresh is triggered during the next cycle by passing a very large number(`math.MaxInt32`) to the `rowsAffected`. +5. Clean up old mutation counts. [automatic_stats.go?L540:7] + +[server_sql.go?L1126:2]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/server/server_sql.go?L1126:2&popover=pinned +[automatic_stats.go?L438:11]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L438:11&popover=pinned +[automatic_stats.go?L518:10]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L518:10&popover=pinned +[automatic_stats.go?L817:3]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L817:3&popover=pinned +[automatic_stats.go?L836:6]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L836:6&popover=pinned +[automatic_stats.go?L843:14]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L843:14&popover=pinned +[automatic_stats.go?L540:7]: https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L540:7&popover=pinned + +#### Frequently Asked Questions + +1. Who can send the mutation count to the refresher? + + `NotifyMutation` is called by SQL mutation operations to signal the Refresher that a table has been mutated. Method [NotifyMutation (automatic_stats.go?L729:21)](https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/sql/stats/automatic_stats.go?L727:21&popover=pinned) + +2. How do they determine the analyzing conflicts? + + They store all the analysis jobs in their database. + So they can check whether there are any other CreateStats jobs in the pending, running, or paused status that started earlier than this one. + +3. Which node will execute the analysis job? + + Because CRDB has a scheduled job framework, it depends on the executor and scheduler. + + For an auto-analysis job, it has an inline executor, it simply executes the job's SQL in a txn. Method [ExecuteJob (executor_impl.go?L40:38)](https://sourcegraph.com/github.com/cockroachdb/cockroach@6d7d8415f112fc478779bf2868be2385237030da/-/blob/pkg/jobs/executor_impl.go?L40:38&popover=pinned) + + The scheduler logic: In short, each node will start a scheduled job execution daemon to attempt to retrieve an executable task from the job queue for execution. Each node has a maximum number of runnable tasks. + +### MySQL + +#### Basic Idea + +The [innodb_stats_auto_recalc](https://dev.mysql.com/doc/refman/8.0/en/innodb-parameters.html#sysvar_innodb_stats_auto_recalc) variable, which is enabled by default, controls whether statistics are calculated automatically when a table undergoes changes to more than 10% of its rows. You can also configure automatic statistics recalculation for individual tables by specifying the `STATS_AUTO_RECALC` clause when creating or altering a table. + +#### Implementation Details + +1. It uses a `recalc_pool` to store tables that need to be processed by background statistics gathering. [dict0stats_bg.cc?L87:23] +2. Call `row_update_statistics_if_needed` when updating data. [row0mysql.cc?L1101:20] + 1. It uses `stat_modified_counter` to indicate how many rows have been modified since the last stats recalc. When a row is inserted, updated, or deleted, it adds to this number. + 2. If `counter` > `n_rows` / 10 (10%), then it pushes the table into the `recalc_pool`. [row0mysql.cc?L1119:9] + 3. Call `dict_stats_recalc_pool_add` to add a table into `recalc_pool`. [dict0stats_bg.cc?L117:6] +3. A thread named `dict_stats_thread` is created to collect statistics in the background. [dict0stats_bg.cc?L355:6] +4. The stats thread is notified by `dict_stats_event`, it is set by `dict_stats_recalc_pool_add`. [dict0stats_bg.cc?L137:16] +5. It also wakes up periodically even if not signaled. [dict0stats_bg.cc?L365:5] +6. After it is notified, it calls `dict_stats_process_entry_from_recalc_pool` to get a table from the pool to recalculate the stats. [dict0stats_bg.cc?L261:13] + 1. If there are a lot of small hot tables, it puts them back and picks another one in the next round. + +[dict0stats_bg.cc?L87:23]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L87:23&popover=pinned +[row0mysql.cc?L1101:20]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/row/row0mysql.cc?L1101:20&popover=pinned +[row0mysql.cc?L1119:9]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/row/row0mysql.cc?L1119:9&popover=pinned +[dict0stats_bg.cc?L117:6]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L117:6&popover=pinned +[dict0stats_bg.cc?L355:6]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L355:6&popover=pinned +[dict0stats_bg.cc?L137:16]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L137:16&popover=pinned +[dict0stats_bg.cc?L365:5]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L365:5&popover=pinned +[dict0stats_bg.cc?L261:13]: https://sourcegraph.com/github.com/mysql/mysql-server@87307d4ddd88405117e3f1e51323836d57ab1f57/-/blob/storage/innobase/dict/dict0stats_bg.cc?L261:13&popover=pinned + +#### Frequently Asked Questions + +1. How many auto-analysis tasks can we run at the same time on the server? + + It only uses one stats thread, picking one table to run each time. + +### SQL Server + +#### Basic Idea + +When the automatic update statistics option, [AUTO_UPDATE_STATISTICS](https://learn.microsoft.com/en-us/sql/t-sql/statements/alter-database-transact-sql-set-options?view=sql-server-ver16#auto_update_statistics) is ON, the Query Optimizer determines when statistics might be out-of-date and then updates them when they are used by a query. + +Starting with SQL Server 2016 (13.x) and under the database compatibility level 130, the Database Engine also uses a decreasing, dynamic statistics recompilation threshold that adjusts according to the table cardinality at the time statistics were evaluated. + +| Table type | Table cardinality (n) | Recompilation threshold (# modifications) | +|------------------------|-----------------------|-------------------------------------------| +| Temporary | n < 6 | 6 | +| Temporary | 6 <= n <= 500 | 500 | +| Permanent | n <= 500 | 500 | +| Temporary or permanent | n > 500 | MIN (500 + (0.20 *n), SQRT(1,000* n)) | + +For example, if your table contains 2 million rows, then the calculation is the minimum of `500 + (0.20 * 2,000,000) = 400,500` and `SQRT(1,000 * 2,000,000) = 44,721`. This means the statistics will be updated every 44,721 modifications. + +## Unresolved Questions + +1. After adopting a time interval as the indicator, will it prevent us from implementing a strategy of updating the queue instead of rebuilding the entire queue in the future? Because after each change, we need to recalculate the time interval for all tables to determine their priority. + + Perhaps we can tolerate a temporary delay, for example, by updating the entire queue only after encountering 100 updates. diff --git a/docs/design/imgs/analyze-dataflow.png b/docs/design/imgs/analyze-dataflow.png new file mode 100644 index 0000000000000..0e77b5f9956e7 Binary files /dev/null and b/docs/design/imgs/analyze-dataflow.png differ diff --git a/dumpling/export/config.go b/dumpling/export/config.go index b6bad7de7e099..52337ec732601 100644 --- a/dumpling/export/config.go +++ b/dumpling/export/config.go @@ -180,10 +180,11 @@ type Config struct { CollationCompatible string CsvOutputDialect CSVDialect - Labels prometheus.Labels `json:"-"` - PromFactory promutil.Factory `json:"-"` - PromRegistry promutil.Registry `json:"-"` - ExtStorage storage.ExternalStorage `json:"-"` + Labels prometheus.Labels `json:"-"` + PromFactory promutil.Factory `json:"-"` + PromRegistry promutil.Registry `json:"-"` + ExtStorage storage.ExternalStorage `json:"-"` + MinTLSVersion uint16 `json:"-"` IOTotalBytes *atomic.Uint64 Net string @@ -276,10 +277,14 @@ func (conf *Config) GetDriverConfig(db string) *mysql.Config { } else { // Use TLS first. driverCfg.AllowFallbackToPlaintext = true + minTLSVersion := uint16(tls.VersionTLS12) + if conf.MinTLSVersion != 0 { + minTLSVersion = conf.MinTLSVersion + } /* #nosec G402 */ driverCfg.TLS = &tls.Config{ InsecureSkipVerify: true, - MinVersion: tls.VersionTLS12, + MinVersion: minTLSVersion, NextProtos: []string{"h2", "http/1.1"}, // specify `h2` to let Go use HTTP/2. } } @@ -754,6 +759,7 @@ func buildTLSConfig(conf *Config) error { util.WithCertAndKeyPath(conf.Security.CertPath, conf.Security.KeyPath), util.WithCAContent(conf.Security.SSLCABytes), util.WithCertAndKeyContent(conf.Security.SSLCertBytes, conf.Security.SSLKeyBytes), + util.WithMinTLSVersion(conf.MinTLSVersion), ) if err != nil { return errors.Trace(err) diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 9d0964b992b0a..b02396cd8e3c1 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -176,6 +176,7 @@ go_library( "@com_github_tikv_client_go_v2//tikvrpc", "@com_github_tikv_client_go_v2//txnkv/rangetask", "@com_github_tikv_client_go_v2//util", + "@com_github_tikv_pd_client//:client", "@com_github_tikv_pd_client//http", "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_client_v3//concurrency", diff --git a/pkg/ddl/backfilling_dist_executor.go b/pkg/ddl/backfilling_dist_executor.go index fb40a600aac99..84e0aece396af 100644 --- a/pkg/ddl/backfilling_dist_executor.go +++ b/pkg/ddl/backfilling_dist_executor.go @@ -141,10 +141,10 @@ func (s *backfillDistExecutor) Init(ctx context.Context) error { if err != nil { return err } - pdLeaderAddr := d.store.(tikv.Storage).GetRegionCache().PDClient().GetLeaderAddr() + discovery := d.store.(tikv.Storage).GetRegionCache().PDClient().GetServiceDiscovery() // TODO: local backend should be inited when step executor is created. // TODO here we have to use executor ctx to avoid it keeps running when task is canceled. - bc, err := ingest.LitBackCtxMgr.Register(s.BaseTaskExecutor.Ctx(), unique, job.ID, d.etcdCli, pdLeaderAddr, job.ReorgMeta.ResourceGroupName) + bc, err := ingest.LitBackCtxMgr.Register(s.BaseTaskExecutor.Ctx(), unique, job.ID, d.etcdCli, discovery, job.ReorgMeta.ResourceGroupName) if err != nil { return errors.Trace(err) } diff --git a/pkg/ddl/db_integration_test.go b/pkg/ddl/db_integration_test.go index cd99c39e760f4..eb1593d5aac25 100644 --- a/pkg/ddl/db_integration_test.go +++ b/pkg/ddl/db_integration_test.go @@ -1628,7 +1628,16 @@ func TestDefaultColumnWithReplace(t *testing.T) { // insert records tk.MustExec("insert into t(c) values (1),(2),(3)") + // Different UUID values will result in different error code. tk.MustGetErrCode("insert into t1(c) values (1)", errno.ErrTruncatedWrongValue) + _, err := tk.Exec("insert into t1(c) values (1)") + originErr := errors.Cause(err) + tErr, ok := originErr.(*terror.Error) + require.Truef(t, ok, "expect type 'terror.Error', but obtain '%T': %v", originErr, originErr) + sqlErr := terror.ToSQLError(tErr) + if int(sqlErr.Code) != errno.ErrTruncatedWrongValue { + require.Equal(t, errno.ErrDataOutOfRange, int(sqlErr.Code)) + } rows := tk.MustQuery("SELECT c1 from t").Rows() for _, row := range rows { diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 690f5676c854c..d058cf60e0f02 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -67,6 +67,7 @@ import ( "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" kvutil "github.com/tikv/client-go/v2/util" + pd "github.com/tikv/pd/client" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -967,12 +968,12 @@ func runIngestReorgJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, return true, 0, nil } ctx := logutil.WithCategory(w.ctx, "ddl-ingest") - var pdLeaderAddr string + var discovery pd.ServiceDiscovery if d != nil { //nolint:forcetypeassert - pdLeaderAddr = d.store.(tikv.Storage).GetRegionCache().PDClient().GetLeaderAddr() + discovery = d.store.(tikv.Storage).GetRegionCache().PDClient().GetServiceDiscovery() } - bc, err = ingest.LitBackCtxMgr.Register(ctx, allIndexInfos[0].Unique, job.ID, nil, pdLeaderAddr, job.ReorgMeta.ResourceGroupName) + bc, err = ingest.LitBackCtxMgr.Register(ctx, allIndexInfos[0].Unique, job.ID, nil, discovery, job.ReorgMeta.ResourceGroupName) if err != nil { ver, err = convertAddIdxJob2RollbackJob(d, t, job, tbl.Meta(), allIndexInfos, err) return false, ver, errors.Trace(err) @@ -1986,8 +1987,8 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { return err } //nolint:forcetypeassert - pdLeaderAddr := w.store.(tikv.Storage).GetRegionCache().PDClient().GetLeaderAddr() - return checkDuplicateForUniqueIndex(w.ctx, t, reorgInfo, pdLeaderAddr) + discovery := w.store.(tikv.Storage).GetRegionCache().PDClient().GetServiceDiscovery() + return checkDuplicateForUniqueIndex(w.ctx, t, reorgInfo, discovery) } } @@ -2024,7 +2025,7 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { return errors.Trace(err) } -func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo *reorgInfo, pdAddr string) error { +func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo *reorgInfo, discovery pd.ServiceDiscovery) error { var bc ingest.BackendCtx var err error defer func() { @@ -2040,7 +2041,7 @@ func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo if indexInfo.Unique { ctx := logutil.WithCategory(ctx, "ddl-ingest") if bc == nil { - bc, err = ingest.LitBackCtxMgr.Register(ctx, indexInfo.Unique, reorgInfo.ID, nil, pdAddr, reorgInfo.ReorgMeta.ResourceGroupName) + bc, err = ingest.LitBackCtxMgr.Register(ctx, indexInfo.Unique, reorgInfo.ID, nil, discovery, reorgInfo.ReorgMeta.ResourceGroupName) if err != nil { return err } diff --git a/pkg/ddl/ingest/BUILD.bazel b/pkg/ddl/ingest/BUILD.bazel index 3ba3ef213bfc4..fca91d77303f6 100644 --- a/pkg/ddl/ingest/BUILD.bazel +++ b/pkg/ddl/ingest/BUILD.bazel @@ -46,6 +46,7 @@ go_library( "@com_github_pingcap_errors//:errors", "@com_github_pingcap_failpoint//:failpoint", "@com_github_tikv_client_go_v2//util", + "@com_github_tikv_pd_client//:client", "@io_etcd_go_etcd_client_v3//:client", "@io_etcd_go_etcd_client_v3//concurrency", "@org_uber_go_atomic//:atomic", diff --git a/pkg/ddl/ingest/backend_mgr.go b/pkg/ddl/ingest/backend_mgr.go index 794598d889807..d2d5a06a8853b 100644 --- a/pkg/ddl/ingest/backend_mgr.go +++ b/pkg/ddl/ingest/backend_mgr.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/pkg/util/generic" "github.com/pingcap/tidb/pkg/util/logutil" kvutil "github.com/tikv/client-go/v2/util" + pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -34,7 +35,7 @@ import ( // BackendCtxMgr is used to manage the backend context. type BackendCtxMgr interface { CheckAvailable() (bool, error) - Register(ctx context.Context, unique bool, jobID int64, etcdClient *clientv3.Client, pdAddr string, resourceGroupName string) (BackendCtx, error) + Register(ctx context.Context, unique bool, jobID int64, etcdClient *clientv3.Client, pdSvcDiscovery pd.ServiceDiscovery, resourceGroupName string) (BackendCtx, error) Unregister(jobID int64) Load(jobID int64) (BackendCtx, bool) @@ -103,7 +104,14 @@ func (m *litBackendCtxMgr) CheckAvailable() (bool, error) { } // Register creates a new backend and registers it to the backend context. -func (m *litBackendCtxMgr) Register(ctx context.Context, unique bool, jobID int64, etcdClient *clientv3.Client, pdAddr string, resourceGroupName string) (BackendCtx, error) { +func (m *litBackendCtxMgr) Register( + ctx context.Context, + unique bool, + jobID int64, + etcdClient *clientv3.Client, + pdSvcDiscovery pd.ServiceDiscovery, + resourceGroupName string, +) (BackendCtx, error) { bc, exist := m.Load(jobID) if !exist { m.memRoot.RefreshConsumption() @@ -116,8 +124,7 @@ func (m *litBackendCtxMgr) Register(ctx context.Context, unique bool, jobID int6 logutil.Logger(ctx).Warn(LitWarnConfigError, zap.Int64("job ID", jobID), zap.Error(err)) return nil, err } - cfg.Lightning.TiDB.PdAddr = pdAddr - bd, err := createLocalBackend(ctx, cfg, resourceGroupName) + bd, err := createLocalBackend(ctx, cfg, pdSvcDiscovery, resourceGroupName) if err != nil { logutil.Logger(ctx).Error(LitErrCreateBackendFail, zap.Int64("job ID", jobID), zap.Error(err)) return nil, err @@ -136,7 +143,12 @@ func (m *litBackendCtxMgr) Register(ctx context.Context, unique bool, jobID int6 return bc, nil } -func createLocalBackend(ctx context.Context, cfg *Config, resourceGroupName string) (*local.Backend, error) { +func createLocalBackend( + ctx context.Context, + cfg *Config, + pdSvcDiscovery pd.ServiceDiscovery, + resourceGroupName string, +) (*local.Backend, error) { tls, err := cfg.Lightning.ToTLS() if err != nil { logutil.Logger(ctx).Error(LitErrCreateBackendFail, zap.Error(err)) @@ -144,14 +156,11 @@ func createLocalBackend(ctx context.Context, cfg *Config, resourceGroupName stri } logutil.BgLogger().Info("create local backend for adding index", zap.String("category", "ddl-ingest"), zap.String("keyspaceName", cfg.KeyspaceName)) - regionSizeGetter := &local.TableRegionSizeGetterImpl{ - DB: nil, - } // We disable the switch TiKV mode feature for now, // because the impact is not fully tested. var raftKV2SwitchModeDuration time.Duration backendConfig := local.NewBackendConfig(cfg.Lightning, int(LitRLimit), cfg.KeyspaceName, resourceGroupName, kvutil.ExplicitTypeDDL, raftKV2SwitchModeDuration) - return local.NewBackend(ctx, tls, backendConfig, regionSizeGetter) + return local.NewBackend(ctx, tls, backendConfig, pdSvcDiscovery) } const checkpointUpdateInterval = 10 * time.Minute diff --git a/pkg/ddl/ingest/mock.go b/pkg/ddl/ingest/mock.go index c6fb06e006ee5..e8a444ee3a1a9 100644 --- a/pkg/ddl/ingest/mock.go +++ b/pkg/ddl/ingest/mock.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/pkg/sessionctx" "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/util/logutil" + pd "github.com/tikv/pd/client" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" ) @@ -57,7 +58,7 @@ func (m *MockBackendCtxMgr) CheckAvailable() (bool, error) { } // Register implements BackendCtxMgr.Register interface. -func (m *MockBackendCtxMgr) Register(_ context.Context, _ bool, jobID int64, _ *clientv3.Client, _ string, _ string) (BackendCtx, error) { +func (m *MockBackendCtxMgr) Register(_ context.Context, _ bool, jobID int64, _ *clientv3.Client, _ pd.ServiceDiscovery, _ string) (BackendCtx, error) { logutil.BgLogger().Info("mock backend mgr register", zap.Int64("jobID", jobID)) if mockCtx, ok := m.runningJobs[jobID]; ok { return mockCtx, nil diff --git a/pkg/disttask/importinto/planner.go b/pkg/disttask/importinto/planner.go index e2a2810c071e7..9224f6aa355e2 100644 --- a/pkg/disttask/importinto/planner.go +++ b/pkg/disttask/importinto/planner.go @@ -205,11 +205,12 @@ func (*PostProcessSpec) ToSubtaskMeta(planCtx planner.PlanCtx) ([]byte, error) { } subtaskMetas = append(subtaskMetas, &subtaskMeta) } - var localChecksum verify.KVChecksum + localChecksum := verify.NewKVGroupChecksumForAdd() maxIDs := make(map[autoid.AllocatorType]int64, 3) for _, subtaskMeta := range subtaskMetas { - checksum := verify.MakeKVChecksum(subtaskMeta.Checksum.Size, subtaskMeta.Checksum.KVs, subtaskMeta.Checksum.Sum) - localChecksum.Add(&checksum) + for id, c := range subtaskMeta.Checksum { + localChecksum.AddRawGroup(id, c.Size, c.KVs, c.Sum) + } for key, val := range subtaskMeta.MaxIDs { if maxIDs[key] < val { @@ -217,13 +218,17 @@ func (*PostProcessSpec) ToSubtaskMeta(planCtx planner.PlanCtx) ([]byte, error) { } } } + c := localChecksum.GetInnerChecksums() postProcessStepMeta := &PostProcessStepMeta{ - Checksum: Checksum{ - Size: localChecksum.SumSize(), - KVs: localChecksum.SumKVS(), - Sum: localChecksum.Sum(), - }, - MaxIDs: maxIDs, + Checksum: make(map[int64]Checksum, len(c)), + MaxIDs: maxIDs, + } + for id, cksum := range c { + postProcessStepMeta.Checksum[id] = Checksum{ + Size: cksum.SumSize(), + KVs: cksum.SumKVS(), + Sum: cksum.Sum(), + } } return json.Marshal(postProcessStepMeta) } diff --git a/pkg/disttask/importinto/planner_test.go b/pkg/disttask/importinto/planner_test.go index f5b8c1e4c3286..28981ccc57e24 100644 --- a/pkg/disttask/importinto/planner_test.go +++ b/pkg/disttask/importinto/planner_test.go @@ -97,7 +97,7 @@ func TestToPhysicalPlan(t *testing.T) { require.NoError(t, err) require.Equal(t, [][]byte{bs}, subtaskMetas1) - subtaskMeta1.Checksum = Checksum{Size: 1, KVs: 2, Sum: 3} + subtaskMeta1.Checksum = map[int64]Checksum{-1: {Size: 1, KVs: 2, Sum: 3}} bs, err = json.Marshal(subtaskMeta1) require.NoError(t, err) planCtx = planner.PlanCtx{ @@ -112,7 +112,7 @@ func TestToPhysicalPlan(t *testing.T) { }, proto.ImportStepPostProcess) require.NoError(t, err) subtaskMeta2 := PostProcessStepMeta{ - Checksum: Checksum{Size: 1, KVs: 2, Sum: 3}, + Checksum: map[int64]Checksum{-1: {Size: 1, KVs: 2, Sum: 3}}, MaxIDs: map[autoid.AllocatorType]int64{}, } bs, err = json.Marshal(subtaskMeta2) diff --git a/pkg/disttask/importinto/proto.go b/pkg/disttask/importinto/proto.go index 129433504e84a..88a53fbc19986 100644 --- a/pkg/disttask/importinto/proto.go +++ b/pkg/disttask/importinto/proto.go @@ -53,7 +53,7 @@ type ImportStepMeta struct { // this is the engine ID, not the id in tidb_background_subtask table. ID int32 Chunks []Chunk - Checksum Checksum + Checksum map[int64]Checksum // see KVGroupChecksum for definition of map key. Result Result // MaxIDs stores the max id that have been used during encoding for each allocator type. // the max id is same among all allocator types for now, since we're using same base, see @@ -94,8 +94,9 @@ type WriteIngestStepMeta struct { // PostProcessStepMeta is the meta of post process step. type PostProcessStepMeta struct { - // accumulated checksum of all subtasks in import step. - Checksum Checksum + // accumulated checksum of all subtasks in import step. See KVGroupChecksum for + // definition of map key. + Checksum map[int64]Checksum // MaxIDs of max all max-ids of subtasks in import step. MaxIDs map[autoid.AllocatorType]int64 } @@ -110,7 +111,7 @@ type SharedVars struct { Progress *importer.Progress mu sync.Mutex - Checksum *verification.KVChecksum + Checksum *verification.KVGroupChecksum SortedDataMeta *external.SortedKVMeta // SortedIndexMetas is a map from index id to its sorted kv meta. diff --git a/pkg/disttask/importinto/subtask_executor.go b/pkg/disttask/importinto/subtask_executor.go index eaab3656d0b85..5202dc70d02d9 100644 --- a/pkg/disttask/importinto/subtask_executor.go +++ b/pkg/disttask/importinto/subtask_executor.go @@ -70,19 +70,38 @@ func (e *importMinimalTaskExecutor) Run(ctx context.Context, dataWriter, indexWr }) chunkCheckpoint := toChunkCheckpoint(e.mTtask.Chunk) sharedVars := e.mTtask.SharedVars + checksum := verify.NewKVGroupChecksumWithKeyspace(sharedVars.TableImporter.GetCodec()) if sharedVars.TableImporter.IsLocalSort() { - if err := importer.ProcessChunk(ctx, &chunkCheckpoint, sharedVars.TableImporter, sharedVars.DataEngine, sharedVars.IndexEngine, sharedVars.Progress, logger); err != nil { + if err := importer.ProcessChunk( + ctx, + &chunkCheckpoint, + sharedVars.TableImporter, + sharedVars.DataEngine, + sharedVars.IndexEngine, + sharedVars.Progress, + logger, + checksum, + ); err != nil { return err } } else { - if err := importer.ProcessChunkWith(ctx, &chunkCheckpoint, sharedVars.TableImporter, dataWriter, indexWriter, sharedVars.Progress, logger); err != nil { + if err := importer.ProcessChunkWithWriter( + ctx, + &chunkCheckpoint, + sharedVars.TableImporter, + dataWriter, + indexWriter, + sharedVars.Progress, + logger, + checksum, + ); err != nil { return err } } sharedVars.mu.Lock() defer sharedVars.mu.Unlock() - sharedVars.Checksum.Add(&chunkCheckpoint.Checksum) + sharedVars.Checksum.Add(checksum) return nil } @@ -109,13 +128,24 @@ func postProcess(ctx context.Context, taskMeta *TaskMeta, subtaskMeta *PostProce // err = multierr.Append(err, err2) // }() - localChecksum := verify.MakeKVChecksum(subtaskMeta.Checksum.Size, subtaskMeta.Checksum.KVs, subtaskMeta.Checksum.Sum) + localChecksum := verify.NewKVGroupChecksumForAdd() + for id, cksum := range subtaskMeta.Checksum { + callLog.Info( + "kv group checksum", + zap.Int64("groupId", id), + zap.Uint64("size", cksum.Size), + zap.Uint64("kvs", cksum.KVs), + zap.Uint64("checksum", cksum.Sum), + ) + localChecksum.AddRawGroup(id, cksum.Size, cksum.KVs, cksum.Sum) + } + taskManager, err := storage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return err } return taskManager.WithNewSession(func(se sessionctx.Context) error { - return importer.VerifyChecksum(ctx, &taskMeta.Plan, localChecksum, se, logger) + return importer.VerifyChecksum(ctx, &taskMeta.Plan, localChecksum.MergedChecksum(), se, logger) }) } diff --git a/pkg/disttask/importinto/task_executor.go b/pkg/disttask/importinto/task_executor.go index 181c4ca4c2d1b..24b86b2e82454 100644 --- a/pkg/disttask/importinto/task_executor.go +++ b/pkg/disttask/importinto/task_executor.go @@ -148,7 +148,7 @@ func (s *importStepExecutor) RunSubtask(ctx context.Context, subtask *proto.Subt DataEngine: dataEngine, IndexEngine: indexEngine, Progress: importer.NewProgress(), - Checksum: &verification.KVChecksum{}, + Checksum: verification.NewKVGroupChecksumWithKeyspace(s.tableImporter.GetCodec()), SortedDataMeta: &external.SortedKVMeta{}, SortedIndexMetas: make(map[int64]*external.SortedKVMeta), } @@ -201,7 +201,7 @@ func (s *importStepExecutor) OnFinished(ctx context.Context, subtask *proto.Subt return errors.Errorf("sharedVars %d not found", subtaskMeta.ID) } - var dataKVCount int64 + var dataKVCount uint64 if s.tableImporter.IsLocalSort() { // TODO: we should close and cleanup engine in all case, since there's no checkpoint. s.logger.Info("import data engine", zap.Int32("engine-id", subtaskMeta.ID)) @@ -209,10 +209,11 @@ func (s *importStepExecutor) OnFinished(ctx context.Context, subtask *proto.Subt if err != nil { return err } - dataKVCount, err = s.tableImporter.ImportAndCleanup(ctx, closedDataEngine) + dataKVCount2, err := s.tableImporter.ImportAndCleanup(ctx, closedDataEngine) if err != nil { return err } + dataKVCount = uint64(dataKVCount2) s.logger.Info("import index engine", zap.Int32("engine-id", subtaskMeta.ID)) if closedEngine, err := sharedVars.IndexEngine.Close(ctx); err != nil { @@ -225,11 +226,16 @@ func (s *importStepExecutor) OnFinished(ctx context.Context, subtask *proto.Subt sharedVars.mu.Lock() defer sharedVars.mu.Unlock() - subtaskMeta.Checksum.Sum = sharedVars.Checksum.Sum() - subtaskMeta.Checksum.KVs = sharedVars.Checksum.SumKVS() - subtaskMeta.Checksum.Size = sharedVars.Checksum.SumSize() + subtaskMeta.Checksum = map[int64]Checksum{} + for id, c := range sharedVars.Checksum.GetInnerChecksums() { + subtaskMeta.Checksum[id] = Checksum{ + Sum: c.Sum(), + KVs: c.SumKVS(), + Size: c.SumSize(), + } + } subtaskMeta.Result = Result{ - LoadedRowCnt: uint64(dataKVCount), + LoadedRowCnt: dataKVCount, ColSizeMap: sharedVars.Progress.GetColSize(), } allocators := sharedVars.TableImporter.Allocators() diff --git a/pkg/disttask/importinto/task_executor_testkit_test.go b/pkg/disttask/importinto/task_executor_testkit_test.go index 3458dbeef69a0..49924751fb5de 100644 --- a/pkg/disttask/importinto/task_executor_testkit_test.go +++ b/pkg/disttask/importinto/task_executor_testkit_test.go @@ -46,10 +46,12 @@ func TestPostProcessStepExecutor(t *testing.T) { tk.MustExec("insert into t values (1, 2), (3, 4)") res := tk.MustQuery("admin checksum table t").Rows() stepMeta := &importinto.PostProcessStepMeta{ - Checksum: importinto.Checksum{ - Sum: uint64(asInt(res[0][2].(string))), - KVs: uint64(asInt(res[0][3].(string))), - Size: uint64(asInt(res[0][4].(string))), + Checksum: map[int64]importinto.Checksum{ + -1: { + Sum: uint64(asInt(res[0][2].(string))), + KVs: uint64(asInt(res[0][3].(string))), + Size: uint64(asInt(res[0][4].(string))), + }, }, } @@ -72,7 +74,9 @@ func TestPostProcessStepExecutor(t *testing.T) { err = executor.RunSubtask(context.Background(), &proto.Subtask{Meta: bytes}) require.NoError(t, err) - stepMeta.Checksum.Sum += 1 + tmp := stepMeta.Checksum[-1] + tmp.Sum += 1 + stepMeta.Checksum[-1] = tmp bytes, err = json.Marshal(stepMeta) require.NoError(t, err) executor = importinto.NewPostProcessStepExecutor(1, taskMeta, zap.NewExample()) diff --git a/pkg/executor/importer/chunk_process.go b/pkg/executor/importer/chunk_process.go index 636f1048212f9..609e010dc9d19 100644 --- a/pkg/executor/importer/chunk_process.go +++ b/pkg/executor/importer/chunk_process.go @@ -48,101 +48,172 @@ var ( MinDeliverRowCnt = 4096 ) -type deliveredRow struct { - kvs *kv.Pairs // if kvs is nil, this indicated we've got the last message. - // offset is the end offset in data file after encode this row. - offset int64 +type rowToEncode struct { + row []types.Datum + rowID int64 + // endOffset represents the offset after the current row in encode reader. + // it will be negative if the data source is not file. + endOffset int64 + resetFn func() } -type deliverKVBatch struct { - dataKVs kv.Pairs - indexKVs kv.Pairs +type encodeReaderFn func(ctx context.Context) (data rowToEncode, closed bool, err error) - dataChecksum *verify.KVChecksum - indexChecksum *verify.KVChecksum +// parserEncodeReader wraps a mydump.Parser as a encodeReaderFn. +func parserEncodeReader(parser mydump.Parser, endOffset int64, filename string) encodeReaderFn { + return func(context.Context) (data rowToEncode, closed bool, err error) { + readPos, _ := parser.Pos() + if readPos >= endOffset { + closed = true + return + } - codec tikv.Codec + err = parser.ReadRow() + // todo: we can implement a ScannedPos which don't return error, will change it later. + currOffset, _ := parser.ScannedPos() + switch errors.Cause(err) { + case nil: + case io.EOF: + closed = true + err = nil + return + default: + err = common.ErrEncodeKV.Wrap(err).GenWithStackByArgs(filename, currOffset) + return + } + lastRow := parser.LastRow() + data = rowToEncode{ + row: lastRow.Row, + rowID: lastRow.RowID, + endOffset: currOffset, + resetFn: func() { parser.RecycleRow(lastRow) }, + } + return + } } -func newDeliverKVBatch(codec tikv.Codec) *deliverKVBatch { - return &deliverKVBatch{ - dataChecksum: verify.NewKVChecksumWithKeyspace(codec), - indexChecksum: verify.NewKVChecksumWithKeyspace(codec), - codec: codec, +// queryRowEncodeReader wraps a QueryRow channel as a encodeReaderFn. +func queryRowEncodeReader(rowCh <-chan QueryRow) encodeReaderFn { + return func(ctx context.Context) (data rowToEncode, closed bool, err error) { + select { + case <-ctx.Done(): + err = ctx.Err() + return + case row, ok := <-rowCh: + if !ok { + closed = true + return + } + data = rowToEncode{ + row: row.Data, + rowID: row.ID, + endOffset: -1, + resetFn: func() {}, + } + return + } } } -func (b *deliverKVBatch) reset() { - b.dataKVs.Clear() - b.indexKVs.Clear() - b.dataChecksum = verify.NewKVChecksumWithKeyspace(b.codec) - b.indexChecksum = verify.NewKVChecksumWithKeyspace(b.codec) +type encodedKVGroupBatch struct { + dataKVs []common.KvPair + indexKVs map[int64][]common.KvPair // indexID -> pairs + + bytesBuf *kv.BytesBuf + memBuf *kv.MemBuf + + groupChecksum *verify.KVGroupChecksum } -func (b *deliverKVBatch) size() uint64 { - return b.dataChecksum.SumSize() + b.indexChecksum.SumSize() +func (b *encodedKVGroupBatch) reset() { + if b.memBuf == nil { + return + } + // mimic kv.Pairs.Clear + b.memBuf.Recycle(b.bytesBuf) + b.bytesBuf = nil + b.memBuf = nil } -func (b *deliverKVBatch) add(kvs *kv.Pairs) { +func newEncodedKVGroupBatch(codec tikv.Codec) *encodedKVGroupBatch { + return &encodedKVGroupBatch{ + indexKVs: make(map[int64][]common.KvPair, 8), + groupChecksum: verify.NewKVGroupChecksumWithKeyspace(codec), + } +} + +// add must be called with `kvs` from the same session for a encodedKVGroupBatch. +func (b *encodedKVGroupBatch) add(kvs *kv.Pairs) error { for _, pair := range kvs.Pairs { if tablecodec.IsRecordKey(pair.Key) { - b.dataKVs.Pairs = append(b.dataKVs.Pairs, pair) - b.dataChecksum.UpdateOne(pair) + b.dataKVs = append(b.dataKVs, pair) + b.groupChecksum.UpdateOneDataKV(pair) } else { - b.indexKVs.Pairs = append(b.indexKVs.Pairs, pair) - b.indexChecksum.UpdateOne(pair) + indexID, err := tablecodec.DecodeIndexID(pair.Key) + if err != nil { + return errors.Trace(err) + } + b.indexKVs[indexID] = append(b.indexKVs[indexID], pair) + b.groupChecksum.UpdateOneIndexKV(indexID, pair) } } - // the related buf is shared, so we only need to set it into one of the kvs so it can be released - if kvs.BytesBuf != nil { - b.dataKVs.BytesBuf = kvs.BytesBuf - b.dataKVs.MemBuf = kvs.MemBuf + // the related buf is shared, so we only need to record any one of them. + if b.bytesBuf == nil && kvs.BytesBuf != nil { + b.bytesBuf = kvs.BytesBuf + b.memBuf = kvs.MemBuf } + return nil } -type chunkEncoder interface { - init() error - encodeLoop(ctx context.Context) error - summaryFields() []zap.Field -} +// chunkEncoder encodes data from readFn and sends encoded data to sendFn. +type chunkEncoder struct { + readFn encodeReaderFn + offset int64 + sendFn func(ctx context.Context, batch *encodedKVGroupBatch) error -// fileChunkEncoder encode data chunk(either a data file or part of a file). -type fileChunkEncoder struct { - parser mydump.Parser - chunkInfo *checkpoints.ChunkCheckpoint + chunkName string logger *zap.Logger encoder KVEncoder kvCodec tikv.Codec - sendFn func(ctx context.Context, kvs []deliveredRow) error - // startOffset is the offset of current source file reader. it might be - // larger than the pos that has been parsed due to reader buffering. - // some rows before startOffset might be skipped if skip_rows > 0. - startOffset int64 - // total duration takes by read/encode/deliver. + // total duration takes by read/encode. readTotalDur time.Duration encodeTotalDur time.Duration -} -var _ chunkEncoder = (*fileChunkEncoder)(nil) + groupChecksum *verify.KVGroupChecksum +} -func (p *fileChunkEncoder) init() error { - // we might skip N rows or start from checkpoint - offset, err := p.parser.ScannedPos() - if err != nil { - return errors.Trace(err) +func newChunkEncoder( + chunkName string, + readFn encodeReaderFn, + offset int64, + sendFn func(ctx context.Context, batch *encodedKVGroupBatch) error, + logger *zap.Logger, + encoder KVEncoder, + kvCodec tikv.Codec, +) *chunkEncoder { + return &chunkEncoder{ + chunkName: chunkName, + readFn: readFn, + offset: offset, + sendFn: sendFn, + logger: logger, + encoder: encoder, + kvCodec: kvCodec, + groupChecksum: verify.NewKVGroupChecksumWithKeyspace(kvCodec), } - p.startOffset = offset - return nil } -func (p *fileChunkEncoder) encodeLoop(ctx context.Context) error { - var err error - reachEOF := false - prevOffset, currOffset := p.startOffset, p.startOffset - - var encodedBytesCounter, encodedRowsCounter prometheus.Counter +func (p *chunkEncoder) encodeLoop(ctx context.Context) error { + var ( + encodedBytesCounter, encodedRowsCounter prometheus.Counter + readDur, encodeDur time.Duration + rowCount int + rowBatch = make([]*kv.Pairs, 0, MinDeliverRowCnt) + rowBatchByteSize uint64 + currOffset int64 + ) metrics, _ := metric.GetCommonMetric(ctx) if metrics != nil { encodedBytesCounter = metrics.BytesCounter.WithLabelValues(metric.StateRestored) @@ -150,85 +221,90 @@ func (p *fileChunkEncoder) encodeLoop(ctx context.Context) error { encodedRowsCounter = metrics.RowsCounter.WithLabelValues(metric.StateRestored, "") } - for !reachEOF { - readPos, _ := p.parser.Pos() - if readPos >= p.chunkInfo.Chunk.EndOffset { - break + recordSendReset := func() error { + if len(rowBatch) == 0 { + return nil } - var readDur, encodeDur time.Duration - canDeliver := false - rowBatch := make([]deliveredRow, 0, MinDeliverRowCnt) - var rowCount, kvSize uint64 - outLoop: - for !canDeliver { - readDurStart := time.Now() - err = p.parser.ReadRow() - readPos, _ = p.parser.Pos() - // todo: we can implement a ScannedPos which don't return error, will change it later. - currOffset, _ = p.parser.ScannedPos() - - switch errors.Cause(err) { - case nil: - case io.EOF: - reachEOF = true - break outLoop - default: - return common.ErrEncodeKV.Wrap(err).GenWithStackByArgs(p.chunkInfo.GetKey(), currOffset) - } - readDur += time.Since(readDurStart) - encodeDurStart := time.Now() - lastRow := p.parser.LastRow() - // sql -> kv - kvs, encodeErr := p.encoder.Encode(lastRow.Row, lastRow.RowID) - encodeDur += time.Since(encodeDurStart) - - if encodeErr != nil { - // todo: record and ignore encode error if user set max-errors param - err = common.ErrEncodeKV.Wrap(encodeErr).GenWithStackByArgs(p.chunkInfo.GetKey(), currOffset) - } - p.parser.RecycleRow(lastRow) - if err != nil { - return err - } - - rowBatch = append(rowBatch, deliveredRow{kvs: kvs, offset: currOffset}) - kvSize += kvs.Size() - rowCount++ - // pebble cannot allow > 4.0G kv in one batch. - // we will meet pebble panic when import sql file and each kv has the size larger than 4G / maxKvPairsCnt. - // so add this check. - if kvSize >= MinDeliverBytes || len(rowBatch) >= MinDeliverRowCnt || readPos == p.chunkInfo.Chunk.EndOffset { - canDeliver = true - } + if currOffset >= 0 && metrics != nil { + delta := currOffset - p.offset + p.offset = currOffset + // if we're using split_file, this metric might larger than total + // source file size, as the offset we're using is the reader offset, + // not parser offset, and we'll buffer data. + encodedBytesCounter.Add(float64(delta)) } - delta := currOffset - prevOffset - prevOffset = currOffset - - p.encodeTotalDur += encodeDur - p.readTotalDur += readDur if metrics != nil { metrics.RowEncodeSecondsHistogram.Observe(encodeDur.Seconds()) metrics.RowReadSecondsHistogram.Observe(readDur.Seconds()) - // if we're using split_file, this metric might larger than total - // source file size, as the offset we're using is the reader offset, - // not parser offset, and we'll buffer data. - encodedBytesCounter.Add(float64(delta)) encodedRowsCounter.Add(float64(rowCount)) } + p.encodeTotalDur += encodeDur + p.readTotalDur += readDur - if len(rowBatch) > 0 { - if err = p.sendFn(ctx, rowBatch); err != nil { + kvGroupBatch := newEncodedKVGroupBatch(p.kvCodec) + + for _, kvs := range rowBatch { + if err := kvGroupBatch.add(kvs); err != nil { + return errors.Trace(err) + } + } + + p.groupChecksum.Add(kvGroupBatch.groupChecksum) + + if err := p.sendFn(ctx, kvGroupBatch); err != nil { + return err + } + + // the ownership of rowBatch is transferred to the receiver of sendFn, we should + // not touch it anymore. + rowBatch = make([]*kv.Pairs, 0, MinDeliverRowCnt) + rowBatchByteSize = 0 + rowCount = 0 + readDur = 0 + encodeDur = 0 + return nil + } + + for { + readDurStart := time.Now() + data, closed, err := p.readFn(ctx) + if err != nil { + return errors.Trace(err) + } + if closed { + break + } + readDur += time.Since(readDurStart) + + encodeDurStart := time.Now() + kvs, encodeErr := p.encoder.Encode(data.row, data.rowID) + currOffset = data.endOffset + data.resetFn() + if encodeErr != nil { + // todo: record and ignore encode error if user set max-errors param + return common.ErrEncodeKV.Wrap(encodeErr).GenWithStackByArgs(p.chunkName, data.endOffset) + } + encodeDur += time.Since(encodeDurStart) + + rowCount++ + rowBatch = append(rowBatch, kvs) + rowBatchByteSize += kvs.Size() + // pebble cannot allow > 4.0G kv in one batch. + // we will meet pebble panic when import sql file and each kv has the size larger than 4G / maxKvPairsCnt. + // so add this check. + if rowBatchByteSize >= MinDeliverBytes || len(rowBatch) >= MinDeliverRowCnt { + if err := recordSendReset(); err != nil { return err } } } - return nil + return recordSendReset() } -func (p *fileChunkEncoder) summaryFields() []zap.Field { +func (p *chunkEncoder) summaryFields() []zap.Field { return []zap.Field{ zap.Duration("readDur", p.readTotalDur), zap.Duration("encodeDur", p.encodeTotalDur), @@ -242,26 +318,23 @@ type ChunkProcessor interface { } type baseChunkProcessor struct { - sourceType DataSourceType - enc chunkEncoder - deliver *dataDeliver - logger *zap.Logger - chunkInfo *checkpoints.ChunkCheckpoint + sourceType DataSourceType + enc *chunkEncoder + deliver *dataDeliver + logger *zap.Logger + groupChecksum *verify.KVGroupChecksum } func (p *baseChunkProcessor) Process(ctx context.Context) (err error) { task := log.BeginTask(p.logger, "process chunk") defer func() { - logFields := append(p.enc.summaryFields(), p.deliver.logFields()...) + logFields := append(p.enc.summaryFields(), p.deliver.summaryFields()...) logFields = append(logFields, zap.Stringer("type", p.sourceType)) task.End(zap.ErrorLevel, err, logFields...) if metrics, ok := metric.GetCommonMetric(ctx); ok && err == nil { metrics.ChunkCounter.WithLabelValues(metric.ChunkStateFinished).Inc() } }() - if err2 := p.enc.init(); err2 != nil { - return err2 - } group, gCtx := errgroup.WithContext(ctx) group.Go(func() error { @@ -273,7 +346,10 @@ func (p *baseChunkProcessor) Process(ctx context.Context) (err error) { }) err2 := group.Wait() - p.chunkInfo.Checksum.Add(&p.deliver.checksum) + // in some unit tests it's nil + if c := p.groupChecksum; c != nil { + c.Add(p.enc.groupChecksum) + } return err2 } @@ -288,51 +364,52 @@ func NewFileChunkProcessor( diskQuotaLock *syncutil.RWMutex, dataWriter backend.EngineWriter, indexWriter backend.EngineWriter, + groupChecksum *verify.KVGroupChecksum, ) ChunkProcessor { chunkLogger := logger.With(zap.String("key", chunk.GetKey())) deliver := &dataDeliver{ logger: chunkLogger, kvCodec: kvCodec, diskQuotaLock: diskQuotaLock, - kvsCh: make(chan []deliveredRow, maxKVQueueSize), + kvBatch: make(chan *encodedKVGroupBatch, maxKVQueueSize), dataWriter: dataWriter, indexWriter: indexWriter, } return &baseChunkProcessor{ sourceType: DataSourceTypeFile, deliver: deliver, - enc: &fileChunkEncoder{ - parser: parser, - chunkInfo: chunk, - logger: chunkLogger, - encoder: encoder, - kvCodec: kvCodec, - sendFn: deliver.sendEncodedData, - }, - logger: chunkLogger, - chunkInfo: chunk, + enc: newChunkEncoder( + chunk.GetKey(), + parserEncodeReader(parser, chunk.Chunk.EndOffset, chunk.GetKey()), + chunk.Chunk.Offset, + deliver.sendEncodedData, + chunkLogger, + encoder, + kvCodec, + ), + logger: chunkLogger, + groupChecksum: groupChecksum, } } type dataDeliver struct { logger *zap.Logger kvCodec tikv.Codec - kvsCh chan []deliveredRow + kvBatch chan *encodedKVGroupBatch diskQuotaLock *syncutil.RWMutex dataWriter backend.EngineWriter indexWriter backend.EngineWriter - checksum verify.KVChecksum deliverTotalDur time.Duration } func (p *dataDeliver) encodeDone() { - close(p.kvsCh) + close(p.kvBatch) } -func (p *dataDeliver) sendEncodedData(ctx context.Context, kvs []deliveredRow) error { +func (p *dataDeliver) sendEncodedData(ctx context.Context, batch *encodedKVGroupBatch) error { select { - case p.kvsCh <- kvs: + case p.kvBatch <- batch: return nil case <-ctx.Done(): return ctx.Err() @@ -340,8 +417,6 @@ func (p *dataDeliver) sendEncodedData(ctx context.Context, kvs []deliveredRow) e } func (p *dataDeliver) deliverLoop(ctx context.Context) error { - kvBatch := newDeliverKVBatch(p.kvCodec) - var ( dataKVBytesHist, indexKVBytesHist prometheus.Observer dataKVPairsHist, indexKVPairsHist prometheus.Observer @@ -358,23 +433,18 @@ func (p *dataDeliver) deliverLoop(ctx context.Context) error { } for { - outer: - for kvBatch.size() < MinDeliverBytes { - select { - case kvPacket, ok := <-p.kvsCh: - if !ok { - break outer - } - for _, row := range kvPacket { - kvBatch.add(row.kvs) - } - case <-ctx.Done(): - return ctx.Err() + var ( + kvBatch *encodedKVGroupBatch + ok bool + ) + + select { + case kvBatch, ok = <-p.kvBatch: + if !ok { + return nil } - } - - if kvBatch.size() == 0 { - break + case <-ctx.Done(): + return ctx.Err() } err := func() error { @@ -382,13 +452,13 @@ func (p *dataDeliver) deliverLoop(ctx context.Context) error { defer p.diskQuotaLock.RUnlock() start := time.Now() - if err := p.dataWriter.AppendRows(ctx, nil, &kvBatch.dataKVs); err != nil { + if err := p.dataWriter.AppendRows(ctx, nil, kv.MakeRowsFromKvPairs(kvBatch.dataKVs)); err != nil { if !common.IsContextCanceledError(err) { p.logger.Error("write to data engine failed", log.ShortError(err)) } return errors.Trace(err) } - if err := p.indexWriter.AppendRows(ctx, nil, &kvBatch.indexKVs); err != nil { + if err := p.indexWriter.AppendRows(ctx, nil, kv.GroupedPairs(kvBatch.indexKVs)); err != nil { if !common.IsContextCanceledError(err) { p.logger.Error("write to index engine failed", log.ShortError(err)) } @@ -399,10 +469,14 @@ func (p *dataDeliver) deliverLoop(ctx context.Context) error { p.deliverTotalDur += deliverDur if metrics != nil { metrics.BlockDeliverSecondsHistogram.Observe(deliverDur.Seconds()) - dataKVBytesHist.Observe(float64(kvBatch.dataChecksum.SumSize())) - indexKVBytesHist.Observe(float64(kvBatch.indexChecksum.SumSize())) - dataKVPairsHist.Observe(float64(kvBatch.dataChecksum.SumKVS())) - indexKVPairsHist.Observe(float64(kvBatch.indexChecksum.SumKVS())) + + dataSize, indexSize := kvBatch.groupChecksum.DataAndIndexSumSize() + dataKVCnt, indexKVCnt := kvBatch.groupChecksum.DataAndIndexSumKVS() + dataKVBytesHist.Observe(float64(dataSize)) + dataKVPairsHist.Observe(float64(dataKVCnt)) + indexKVBytesHist.Observe(float64(indexSize)) + indexKVPairsHist.Observe(float64(indexKVCnt)) + deliverBytesCounter.Add(float64(dataSize + indexSize)) } return nil }() @@ -410,124 +484,13 @@ func (p *dataDeliver) deliverLoop(ctx context.Context) error { return err } - if metrics != nil { - deliverBytesCounter.Add(float64(kvBatch.size())) - } - - p.checksum.Add(kvBatch.dataChecksum) - p.checksum.Add(kvBatch.indexChecksum) - kvBatch.reset() } - - return nil } -func (p *dataDeliver) logFields() []zap.Field { +func (p *dataDeliver) summaryFields() []zap.Field { return []zap.Field{ zap.Duration("deliverDur", p.deliverTotalDur), - zap.Object("checksum", &p.checksum), - } -} - -// fileChunkEncoder encode data chunk(either a data file or part of a file). -type queryChunkEncoder struct { - rowCh chan QueryRow - chunkInfo *checkpoints.ChunkCheckpoint - logger *zap.Logger - encoder KVEncoder - sendFn func(ctx context.Context, kvs []deliveredRow) error - - // total duration takes by read/encode/deliver. - readTotalDur time.Duration - encodeTotalDur time.Duration -} - -var _ chunkEncoder = (*queryChunkEncoder)(nil) - -func (*queryChunkEncoder) init() error { - return nil -} - -// TODO logic is very similar to fileChunkEncoder, consider merge them. -func (e *queryChunkEncoder) encodeLoop(ctx context.Context) error { - var err error - reachEOF := false - var encodedRowsCounter prometheus.Counter - metrics, _ := metric.GetCommonMetric(ctx) - if metrics != nil { - // table name doesn't matter here, all those metrics will have task-id label. - encodedRowsCounter = metrics.RowsCounter.WithLabelValues(metric.StateRestored, "") - } - - for !reachEOF { - var readDur, encodeDur time.Duration - canDeliver := false - rowBatch := make([]deliveredRow, 0, MinDeliverRowCnt) - var rowCount, kvSize uint64 - outLoop: - for !canDeliver { - readDurStart := time.Now() - var ( - lastRow QueryRow - rowID int64 - ok bool - ) - select { - case lastRow, ok = <-e.rowCh: - if !ok { - reachEOF = true - break outLoop - } - case <-ctx.Done(): - return ctx.Err() - } - readDur += time.Since(readDurStart) - encodeDurStart := time.Now() - // sql -> kv - kvs, encodeErr := e.encoder.Encode(lastRow.Data, lastRow.ID) - encodeDur += time.Since(encodeDurStart) - - if encodeErr != nil { - err = common.ErrEncodeKV.Wrap(encodeErr).GenWithStackByArgs(e.chunkInfo.GetKey(), rowID) - } - if err != nil { - return err - } - - rowBatch = append(rowBatch, deliveredRow{kvs: kvs}) - kvSize += kvs.Size() - rowCount++ - // pebble cannot allow > 4.0G kv in one batch. - // we will meet pebble panic when import sql file and each kv has the size larger than 4G / maxKvPairsCnt. - // so add this check. - if kvSize >= MinDeliverBytes || len(rowBatch) >= MinDeliverRowCnt { - canDeliver = true - } - } - - e.encodeTotalDur += encodeDur - e.readTotalDur += readDur - if metrics != nil { - metrics.RowEncodeSecondsHistogram.Observe(encodeDur.Seconds()) - metrics.RowReadSecondsHistogram.Observe(readDur.Seconds()) - encodedRowsCounter.Add(float64(rowCount)) - } - - if len(rowBatch) > 0 { - if err = e.sendFn(ctx, rowBatch); err != nil { - return err - } - } - } - - return nil -} - -func (e *queryChunkEncoder) summaryFields() []zap.Field { - return []zap.Field{ - zap.Duration("readDur", e.readTotalDur), - zap.Duration("encodeDur", e.encodeTotalDur), } } @@ -541,44 +504,42 @@ func newQueryChunkProcessor( rowCh chan QueryRow, encoder KVEncoder, kvCodec tikv.Codec, - chunk *checkpoints.ChunkCheckpoint, logger *zap.Logger, diskQuotaLock *syncutil.RWMutex, dataWriter backend.EngineWriter, indexWriter backend.EngineWriter, + groupChecksum *verify.KVGroupChecksum, ) ChunkProcessor { - chunkLogger := logger.With(zap.String("key", chunk.GetKey())) + chunkName := "import-from-select" + chunkLogger := logger.With(zap.String("key", chunkName)) deliver := &dataDeliver{ logger: chunkLogger, kvCodec: kvCodec, diskQuotaLock: diskQuotaLock, - kvsCh: make(chan []deliveredRow, maxKVQueueSize), + kvBatch: make(chan *encodedKVGroupBatch, maxKVQueueSize), dataWriter: dataWriter, indexWriter: indexWriter, } return &baseChunkProcessor{ sourceType: DataSourceTypeQuery, deliver: deliver, - enc: &queryChunkEncoder{ - rowCh: rowCh, - chunkInfo: chunk, - logger: chunkLogger, - encoder: encoder, - sendFn: deliver.sendEncodedData, - }, - logger: chunkLogger, - chunkInfo: chunk, + enc: newChunkEncoder( + chunkName, + queryRowEncodeReader(rowCh), + -1, + deliver.sendEncodedData, + chunkLogger, + encoder, + kvCodec, + ), + logger: chunkLogger, + groupChecksum: groupChecksum, } } // IndexRouteWriter is a writer for index when using global sort. // we route kvs of different index to different writer in order to make // merge sort easier, else kv data of all subtasks will all be overlapped. -// -// drawback of doing this is that the number of writers need to open will be -// index-count * encode-concurrency, when the table has many indexes, and each -// writer will take 256MiB buffer on default. -// this will take a lot of memory, or even OOM. type IndexRouteWriter struct { writers map[int64]*external.Writer logger *zap.Logger @@ -596,24 +557,23 @@ func NewIndexRouteWriter(logger *zap.Logger, writerFactory func(int64) *external // AppendRows implements backend.EngineWriter interface. func (w *IndexRouteWriter) AppendRows(ctx context.Context, _ []string, rows encode.Rows) error { - kvs := kv.Rows2KvPairs(rows) - if len(kvs) == 0 { - return nil + groupedKvs, ok := rows.(kv.GroupedPairs) + if !ok { + return errors.Errorf("invalid kv pairs type for IndexRouteWriter: %T", rows) } - for _, item := range kvs { - indexID, err := tablecodec.DecodeIndexID(item.Key) - if err != nil { - return errors.Trace(err) - } - writer, ok := w.writers[indexID] - if !ok { - writer = w.writerFactory(indexID) - w.writers[indexID] = writer - } - if err = writer.WriteRow(ctx, item.Key, item.Val, nil); err != nil { - return errors.Trace(err) + for indexID, kvs := range groupedKvs { + for _, item := range kvs { + writer, ok := w.writers[indexID] + if !ok { + writer = w.writerFactory(indexID) + w.writers[indexID] = writer + } + if err := writer.WriteRow(ctx, item.Key, item.Val, nil); err != nil { + return errors.Trace(err) + } } } + return nil } diff --git a/pkg/executor/importer/chunk_process_testkit_test.go b/pkg/executor/importer/chunk_process_testkit_test.go index f24fc70ee6b8d..7a5f8605d583e 100644 --- a/pkg/executor/importer/chunk_process_testkit_test.go +++ b/pkg/executor/importer/chunk_process_testkit_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/log" "github.com/pingcap/tidb/br/pkg/lightning/metric" "github.com/pingcap/tidb/br/pkg/lightning/mydump" + verify "github.com/pingcap/tidb/br/pkg/lightning/verification" "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/pkg/disttask/framework/proto" "github.com/pingcap/tidb/pkg/executor/importer" @@ -132,21 +133,28 @@ func TestFileChunkProcess(t *testing.T) { }).AnyTimes() indexWriter.EXPECT().AppendRows(gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( func(ctx context.Context, columnNames []string, rows encode.Rows) error { - indexKVCnt += len(rows.(*kv.Pairs).Pairs) + group := rows.(kv.GroupedPairs) + for _, pairs := range group { + indexKVCnt += len(pairs) + } return nil }).AnyTimes() chunkInfo := &checkpoints.ChunkCheckpoint{ Chunk: mydump.Chunk{EndOffset: int64(len(sourceData)), RowIDMax: 10000}, } + checksum := verify.NewKVGroupChecksumWithKeyspace(codec) processor := importer.NewFileChunkProcessor( csvParser, encoder, codec, - chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, + chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, checksum, ) require.NoError(t, processor.Process(ctx)) require.True(t, ctrl.Satisfied()) - require.Equal(t, uint64(9), chunkInfo.Checksum.SumKVS()) - require.Equal(t, uint64(348), chunkInfo.Checksum.SumSize()) + checksumDataKVCnt, checksumIndexKVCnt := checksum.DataAndIndexSumKVS() + require.Equal(t, uint64(3), checksumDataKVCnt) + require.Equal(t, uint64(6), checksumIndexKVCnt) + dataKVSize, indexKVSize := checksum.DataAndIndexSumSize() + require.Equal(t, uint64(348), dataKVSize+indexKVSize) require.Equal(t, 3, dataKVCnt) require.Equal(t, 6, indexKVCnt) require.Equal(t, float64(len(sourceData)), metric.ReadCounter(metrics.BytesCounter.WithLabelValues(metric.StateRestored))) @@ -174,7 +182,7 @@ func TestFileChunkProcess(t *testing.T) { } processor := importer.NewFileChunkProcessor( csvParser, encoder, codec, - chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, + chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, nil, ) require.ErrorIs(t, processor.Process(ctx), common.ErrEncodeKV) require.True(t, ctrl.Satisfied()) @@ -199,7 +207,7 @@ func TestFileChunkProcess(t *testing.T) { } processor := importer.NewFileChunkProcessor( csvParser, encoder, codec, - chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, + chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, nil, ) require.ErrorIs(t, processor.Process(ctx), common.ErrEncodeKV) require.True(t, ctrl.Satisfied()) @@ -223,7 +231,7 @@ func TestFileChunkProcess(t *testing.T) { } processor := importer.NewFileChunkProcessor( csvParser, encoder, codec, - chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, + chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, nil, ) require.ErrorContains(t, processor.Process(ctx), "data write error") require.True(t, ctrl.Satisfied()) @@ -248,7 +256,7 @@ func TestFileChunkProcess(t *testing.T) { } processor := importer.NewFileChunkProcessor( csvParser, encoder, codec, - chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, + chunkInfo, logger.Logger, diskQuotaLock, dataWriter, indexWriter, nil, ) require.ErrorContains(t, processor.Process(ctx), "index write error") require.True(t, ctrl.Satisfied()) diff --git a/pkg/executor/importer/engine_process.go b/pkg/executor/importer/engine_process.go index 51803023ac8fd..6720bdb8e45fe 100644 --- a/pkg/executor/importer/engine_process.go +++ b/pkg/executor/importer/engine_process.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" - "github.com/pingcap/tidb/br/pkg/lightning/mydump" + "github.com/pingcap/tidb/br/pkg/lightning/verification" "go.uber.org/zap" ) @@ -29,10 +29,10 @@ func ProcessChunk( ctx context.Context, chunk *checkpoints.ChunkCheckpoint, tableImporter *TableImporter, - dataEngine, - indexEngine *backend.OpenedEngine, + dataEngine, indexEngine *backend.OpenedEngine, progress *Progress, logger *zap.Logger, + groupChecksum *verification.KVGroupChecksum, ) error { // if the key are ordered, LocalWrite can optimize the writing. // table has auto-incremented _tidb_rowid must satisfy following restrictions: @@ -66,33 +66,19 @@ func ProcessChunk( } }() - return ProcessChunkWith(ctx, chunk, tableImporter, dataWriter, indexWriter, progress, logger) + return ProcessChunkWithWriter(ctx, chunk, tableImporter, dataWriter, indexWriter, progress, logger, groupChecksum) } -// ProcessChunkWith processes a chunk, and write kv pairs to dataWriter and indexWriter. -func ProcessChunkWith( +// ProcessChunkWithWriter processes a chunk, and write kv pairs to dataWriter and indexWriter. +func ProcessChunkWithWriter( ctx context.Context, chunk *checkpoints.ChunkCheckpoint, tableImporter *TableImporter, dataWriter, indexWriter backend.EngineWriter, progress *Progress, logger *zap.Logger, + groupChecksum *verification.KVGroupChecksum, ) error { - var ( - err error - parser mydump.Parser - ) - if tableImporter.DataSourceType == DataSourceTypeFile { - parser, err = tableImporter.getParser(ctx, chunk) - if err != nil { - return err - } - defer func() { - if err2 := parser.Close(); err2 != nil { - logger.Warn("close parser failed", zap.Error(err2)) - } - }() - } encoder, err := tableImporter.getKVEncoder(chunk) if err != nil { return err @@ -106,15 +92,25 @@ func ProcessChunkWith( // TODO: right now we use this chunk processor for global sort too, will // impl another one for it later. var cp ChunkProcessor - if tableImporter.DataSourceType == DataSourceTypeQuery { - cp = newQueryChunkProcessor( - tableImporter.rowCh, encoder, tableImporter.kvStore.GetCodec(), chunk, logger, - tableImporter.diskQuotaLock, dataWriter, indexWriter, - ) - } else { + switch tableImporter.DataSourceType { + case DataSourceTypeFile: + parser, err := tableImporter.getParser(ctx, chunk) + if err != nil { + return err + } + defer func() { + if err2 := parser.Close(); err2 != nil { + logger.Warn("close parser failed", zap.Error(err2)) + } + }() cp = NewFileChunkProcessor( - parser, encoder, tableImporter.kvStore.GetCodec(), chunk, logger, - tableImporter.diskQuotaLock, dataWriter, indexWriter, + parser, encoder, tableImporter.GetCodec(), chunk, logger, + tableImporter.diskQuotaLock, dataWriter, indexWriter, groupChecksum, + ) + case DataSourceTypeQuery: + cp = newQueryChunkProcessor( + tableImporter.rowCh, encoder, tableImporter.GetCodec(), logger, + tableImporter.diskQuotaLock, dataWriter, indexWriter, groupChecksum, ) } err = cp.Process(ctx) diff --git a/pkg/executor/importer/importer_testkit_test.go b/pkg/executor/importer/importer_testkit_test.go index 1495b8334cbcc..7665238bc2916 100644 --- a/pkg/executor/importer/importer_testkit_test.go +++ b/pkg/executor/importer/importer_testkit_test.go @@ -224,10 +224,12 @@ func TestPostProcess(t *testing.T) { logger := zap.NewExample() // verify checksum failed - localChecksum := verify.MakeKVChecksum(1, 2, 1) + localChecksum := verify.NewKVGroupChecksumForAdd() + localChecksum.AddRawGroup(verify.DataKVGroupID, 1, 2, 1) require.ErrorIs(t, importer.PostProcess(ctx, tk.Session(), nil, plan, localChecksum, logger), common.ErrChecksumMismatch) // success - localChecksum = verify.MakeKVChecksum(1, 1, 1) + localChecksum = verify.NewKVGroupChecksumForAdd() + localChecksum.AddRawGroup(verify.DataKVGroupID, 1, 1, 1) require.NoError(t, importer.PostProcess(ctx, tk.Session(), nil, plan, localChecksum, logger)) // get KV store failed importer.GetKVStore = func(path string, tls kvconfig.Security) (kv.Storage, error) { @@ -320,10 +322,13 @@ func TestProcessChunkWith(t *testing.T) { kvWriter := mock.NewMockEngineWriter(ctrl) kvWriter.EXPECT().AppendRows(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() progress := importer.NewProgress() - err := importer.ProcessChunkWith(ctx, chunkInfo, ti, kvWriter, kvWriter, progress, zap.NewExample()) + checksum := verify.NewKVGroupChecksumWithKeyspace(store.GetCodec()) + err := importer.ProcessChunkWithWriter(ctx, chunkInfo, ti, kvWriter, kvWriter, progress, zap.NewExample(), checksum) require.NoError(t, err) require.Len(t, progress.GetColSize(), 3) - require.Equal(t, verify.MakeKVChecksum(74, 2, 15625182175392723123), chunkInfo.Checksum) + checksumMap := checksum.GetInnerChecksums() + require.Len(t, checksumMap, 1) + require.Equal(t, verify.MakeKVChecksum(74, 2, 15625182175392723123), *checksumMap[verify.DataKVGroupID]) }) t.Run("query chunk", func(t *testing.T) { @@ -349,10 +354,13 @@ func TestProcessChunkWith(t *testing.T) { kvWriter := mock.NewMockEngineWriter(ctrl) kvWriter.EXPECT().AppendRows(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() progress := importer.NewProgress() - err := importer.ProcessChunkWith(ctx, chunkInfo, ti, kvWriter, kvWriter, progress, zap.NewExample()) + checksum := verify.NewKVGroupChecksumWithKeyspace(store.GetCodec()) + err := importer.ProcessChunkWithWriter(ctx, chunkInfo, ti, kvWriter, kvWriter, progress, zap.NewExample(), checksum) require.NoError(t, err) require.Len(t, progress.GetColSize(), 3) - require.Equal(t, verify.MakeKVChecksum(111, 3, 14231358899564314836), chunkInfo.Checksum) + checksumMap := checksum.GetInnerChecksums() + require.Len(t, checksumMap, 1) + require.Equal(t, verify.MakeKVChecksum(111, 3, 14231358899564314836), *checksumMap[verify.DataKVGroupID]) }) } diff --git a/pkg/executor/importer/table_import.go b/pkg/executor/importer/table_import.go index 9dd11d1f72e28..5f4f58fce5b77 100644 --- a/pkg/executor/importer/table_import.go +++ b/pkg/executor/importer/table_import.go @@ -61,6 +61,7 @@ import ( "github.com/pingcap/tidb/pkg/util/sqlkiller" "github.com/pingcap/tidb/pkg/util/syncutil" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/util" pdhttp "github.com/tikv/pd/client/http" clientv3 "go.etcd.io/etcd/client/v3" @@ -220,10 +221,8 @@ func NewTableImporter(param *JobImportParam, e *LoadDataController, id string) ( } backendConfig := e.getLocalBackendCfg(tidbCfg.Path, dir) - - // todo: use a real region size getter - regionSizeGetter := &local.TableRegionSizeGetterImpl{} - localBackend, err := local.NewBackend(param.GroupCtx, tls, backendConfig, regionSizeGetter) + d := kvStore.(tikv.Storage).GetRegionCache().PDClient().GetServiceDiscovery() + localBackend, err := local.NewBackend(param.GroupCtx, tls, backendConfig, d) if err != nil { return nil, err } @@ -315,6 +314,11 @@ func NewTableImporterForTest(param *JobImportParam, e *LoadDataController, id st }, nil } +// GetCodec gets the codec of the kv store. +func (ti *TableImporter) GetCodec() tikv.Codec { + return ti.kvStore.GetCodec() +} + func (ti *TableImporter) getParser(ctx context.Context, chunk *checkpoints.ChunkCheckpoint) (mydump.Parser, error) { info := LoadDataReaderInfo{ Opener: func(ctx context.Context) (io.ReadSeekCloser, error) { @@ -690,23 +694,24 @@ func (ti *TableImporter) ImportSelectedRows(ctx context.Context, se sessionctx.C var ( mu sync.Mutex - checksum verify.KVChecksum + checksum = verify.NewKVGroupChecksumWithKeyspace(ti.GetCodec()) colSizeMap = make(map[int64]int64) ) eg, egCtx := tidbutil.NewErrorGroupWithRecoverWithCtx(ctx) for i := 0; i < ti.ThreadCnt; i++ { eg.Go(func() error { chunkCheckpoint := checkpoints.ChunkCheckpoint{} + chunkChecksum := verify.NewKVGroupChecksumWithKeyspace(ti.GetCodec()) progress := NewProgress() defer func() { mu.Lock() defer mu.Unlock() - checksum.Add(&chunkCheckpoint.Checksum) + checksum.Add(chunkChecksum) for k, v := range progress.GetColSize() { colSizeMap[k] += v } }() - return ProcessChunk(egCtx, &chunkCheckpoint, ti, dataEngine, indexEngine, progress, ti.logger) + return ProcessChunk(egCtx, &chunkCheckpoint, ti, dataEngine, indexEngine, progress, ti.logger, chunkChecksum) }) } if err = eg.Wait(); err != nil { @@ -782,10 +787,10 @@ func PostProcess( se sessionctx.Context, maxIDs map[autoid.AllocatorType]int64, plan *Plan, - localChecksum verify.KVChecksum, + localChecksum *verify.KVGroupChecksum, logger *zap.Logger, ) (err error) { - callLog := log.BeginTask(logger.With(zap.Any("checksum", localChecksum)), "post process") + callLog := log.BeginTask(logger.With(zap.Object("checksum", localChecksum)), "post process") defer func() { callLog.End(zap.ErrorLevel, err) }() @@ -794,7 +799,7 @@ func PostProcess( return err } - return VerifyChecksum(ctx, plan, localChecksum, se, logger) + return VerifyChecksum(ctx, plan, localChecksum.MergedChecksum(), se, logger) } type autoIDRequirement struct { diff --git a/pkg/util/security.go b/pkg/util/security.go index af1f3902410a9..c2df44ab10f70 100644 --- a/pkg/util/security.go +++ b/pkg/util/security.go @@ -109,6 +109,7 @@ type tlsConfigBuilder struct { caPath, certPath, keyPath string caContent, certContent, keyContent []byte verifyCN []string + minTLSVersion uint16 } // TLSConfigOption is used to build a tls.Config in NewTLSConfig. @@ -162,6 +163,13 @@ func WithCertAndKeyContent(certContent, keyContent []byte) TLSConfigOption { } } +// WithMinTLSVersion sets the min tls version to build a tls.Config. +func WithMinTLSVersion(minTLSVersion uint16) TLSConfigOption { + return func(builder *tlsConfigBuilder) { + builder.minTLSVersion = minTLSVersion + } +} + // NewTLSConfig creates a tls.Config from the given options. If no certificate is provided, it will return (nil, nil). func NewTLSConfig(opts ...TLSConfigOption) (*tls.Config, error) { builder := &tlsConfigBuilder{} @@ -188,6 +196,10 @@ func NewTLSConfig(opts ...TLSConfigOption) (*tls.Config, error) { NextProtos: []string{"h2", "http/1.2"}, // specify `h2` to let Go use HTTP/2. } + if builder.minTLSVersion != 0 { + tlsCfg.MinVersion = builder.minTLSVersion + } + // 1. handle client certificates if builder.certPath != "" && builder.keyPath != "" { diff --git a/pkg/util/security_test.go b/pkg/util/security_test.go index 92af20563ce36..74f6c44c3b383 100644 --- a/pkg/util/security_test.go +++ b/pkg/util/security_test.go @@ -175,6 +175,15 @@ func TestTLSVersion(t *testing.T) { require.Error(t, err) } } + + // test with min tls version + clientTLS2, err := util.NewTLSConfig( + util.WithCAContent(caData), + util.WithCertAndKeyContent(clientCert, clientKey), + util.WithMinTLSVersion(tls.VersionTLS13), + ) + require.NoError(t, err) + require.Equal(t, uint16(tls.VersionTLS13), clientTLS2.MinVersion) } func TestCA(t *testing.T) {