diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index db60aea6d7871..9367c984b98d3 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -587,16 +587,16 @@ const ( // DupeResAlgNone doesn't detect duplicate. DupeResAlgNone DuplicateResolutionAlgorithm = iota - // DupeResAlgRecord only records duplicate records to `lightning_task_info.conflict_error_v1` table on the target TiDB. + // 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_v1 table to add back the correct rows. + // 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_v1 table to check whether the reserved data cater to their need and check whether + // 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 diff --git a/br/pkg/lightning/errormanager/BUILD.bazel b/br/pkg/lightning/errormanager/BUILD.bazel index d099121c777f2..05d8cb8f6e1f6 100644 --- a/br/pkg/lightning/errormanager/BUILD.bazel +++ b/br/pkg/lightning/errormanager/BUILD.bazel @@ -18,6 +18,7 @@ go_library( "//pkg/table", "//pkg/table/tables", "//pkg/tablecodec", + "//pkg/types", "@com_github_jedib0t_go_pretty_v6//table", "@com_github_jedib0t_go_pretty_v6//text", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/lightning/errormanager/errormanager.go b/br/pkg/lightning/errormanager/errormanager.go index 9c841bb810beb..80d1ab32df5a6 100644 --- a/br/pkg/lightning/errormanager/errormanager.go +++ b/br/pkg/lightning/errormanager/errormanager.go @@ -38,6 +38,7 @@ import ( tidbtbl "github.com/pingcap/tidb/pkg/table" "github.com/pingcap/tidb/pkg/table/tables" "github.com/pingcap/tidb/pkg/tablecodec" + "github.com/pingcap/tidb/pkg/types" tikverr "github.com/tikv/client-go/v2/error" "go.uber.org/atomic" "go.uber.org/multierr" @@ -54,7 +55,7 @@ const ( syntaxErrorTableName = "syntax_error_v1" typeErrorTableName = "type_error_v1" // ConflictErrorTableName is the table name for duplicate detection. - ConflictErrorTableName = "conflict_error_v1" + ConflictErrorTableName = "conflict_error_v2" // DupRecordTable is the table name to record duplicate data that displayed to user. DupRecordTable = "conflict_records" @@ -94,6 +95,7 @@ const ( raw_value mediumblob NOT NULL COMMENT 'the value of the conflicted key', raw_handle mediumblob NOT NULL COMMENT 'the data handle derived from the conflicted key or value', raw_row mediumblob NOT NULL COMMENT 'the data retrieved from the handle', + is_data_kv tinyint(1) NOT NULL, INDEX (task_id, table_name), INDEX (index_name), INDEX (table_name, index_name) @@ -122,19 +124,19 @@ const ( insertIntoConflictErrorData = ` INSERT INTO %s.` + ConflictErrorTableName + ` - (task_id, table_name, index_name, key_data, row_data, raw_key, raw_value, raw_handle, raw_row) + (task_id, table_name, index_name, key_data, row_data, raw_key, raw_value, raw_handle, raw_row, is_data_kv) VALUES ` - sqlValuesConflictErrorData = "(?,?,'PRIMARY',?,?,?,?,raw_key,raw_value)" + sqlValuesConflictErrorData = "(?,?,'PRIMARY',?,?,?,?,raw_key,raw_value,?)" insertIntoConflictErrorIndex = ` INSERT INTO %s.` + ConflictErrorTableName + ` - (task_id, table_name, index_name, key_data, row_data, raw_key, raw_value, raw_handle, raw_row) + (task_id, table_name, index_name, key_data, row_data, raw_key, raw_value, raw_handle, raw_row, is_data_kv) VALUES ` - sqlValuesConflictErrorIndex = "(?,?,?,?,?,?,?,?,?)" + sqlValuesConflictErrorIndex = "(?,?,?,?,?,?,?,?,?,?)" selectConflictKeysRemove = ` SELECT _tidb_rowid, raw_handle, raw_row @@ -146,14 +148,14 @@ const ( selectIndexConflictKeysReplace = ` SELECT raw_key, index_name, raw_value, raw_handle FROM %s.` + ConflictErrorTableName + ` - WHERE table_name = ? AND index_name <> 'PRIMARY' + WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key; ` selectDataConflictKeysReplace = ` - SELECT raw_key, raw_value, raw_handle + SELECT raw_key, raw_value FROM %s.` + ConflictErrorTableName + ` - WHERE table_name = ? AND index_name = 'PRIMARY' + WHERE table_name = ? AND is_data_kv = 1 ORDER BY raw_key; ` @@ -252,7 +254,7 @@ func (em *ErrorManager) Init(ctx context.Context) error { sqls = append(sqls, [2]string{"create type error table", createTypeErrorTable}) } if em.conflictV1Enabled { - sqls = append(sqls, [2]string{"create conflict error v1 table", createConflictErrorTable}) + sqls = append(sqls, [2]string{"create conflict error v2 table", createConflictErrorTable}) } if em.conflictV2Enabled { sqls = append(sqls, [2]string{"create duplicate records table", createDupRecordTable}) @@ -378,6 +380,7 @@ func (em *ErrorManager) RecordDataConflictError( conflictInfo.Row, conflictInfo.RawKey, conflictInfo.RawValue, + tablecodec.IsRecordKey(conflictInfo.RawKey), ) } _, err := txn.ExecContext(c, sb.String(), sqlArgs...) @@ -439,6 +442,7 @@ func (em *ErrorManager) RecordIndexConflictError( conflictInfo.RawValue, rawHandles[i], rawRows[i], + tablecodec.IsRecordKey(conflictInfo.RawKey), ) } _, err := txn.ExecContext(c, sb.String(), sqlArgs...) @@ -561,7 +565,7 @@ func (em *ErrorManager) ReplaceConflictKeys( 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 - // check index KV first + // check index KV indexKvRows, err := em.db.QueryContext( gCtx, fmt.Sprintf(selectIndexConflictKeysReplace, em.schemaEscaped), tableName) @@ -583,7 +587,10 @@ func (em *ErrorManager) ReplaceConflictKeys( // get the latest value of rawKey from downstream TiDB latestValue, err := fnGetLatest(gCtx, rawKey) - if err != nil && !tikverr.IsErrNotFound(err) { + if tikverr.IsErrNotFound(err) { + continue + } + if err != nil { return errors.Trace(err) } @@ -597,7 +604,7 @@ func (em *ErrorManager) ReplaceConflictKeys( // get the latest value of the row key of the data KV that needs to be deleted overwritten, err := fnGetLatest(gCtx, rawHandle) // if the latest value cannot be found, that means the data KV has been deleted - if tikverr.IsErrNotFound(err) || overwritten == nil { + if tikverr.IsErrNotFound(err) { continue } if err != nil { @@ -613,12 +620,24 @@ func (em *ErrorManager) ReplaceConflictKeys( if err != nil { return errors.Trace(err) } + if !tbl.Meta().HasClusteredIndex() { + // for nonclustered PK, need to append handle to decodedData for AddRecord + decodedData = append(decodedData, types.NewIntDatum(overwrittenHandle.IntValue())) + } _, err = encoder.Table.AddRecord(encoder.SessionCtx, decodedData) if err != nil { return errors.Trace(err) } + // 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), @@ -644,6 +663,26 @@ func (em *ErrorManager) ReplaceConflictKeys( // Only if there is a->1 we dare to delete data KV with key "1". if bytes.Equal(kvPair.Key, rawKey) && bytes.Equal(kvPair.Val, rawValue) { + if err := exec.Transact(ctx, "insert data conflict error record for conflict detection 'replace' mode", + func(c context.Context, txn *sql.Tx) error { + sb := &strings.Builder{} + fmt.Fprintf(sb, insertIntoConflictErrorData, em.schemaEscaped) + var sqlArgs []interface{} + sb.WriteString(sqlValuesConflictErrorData) + sqlArgs = append(sqlArgs, + em.taskID, + tableName, + nil, + nil, + rawHandle, + overwritten, + 1, + ) + _, err := txn.ExecContext(c, sb.String(), sqlArgs...) + return err + }); err != nil { + return err + } if err := fnDeleteKey(gCtx, rawHandle); err != nil { return errors.Trace(err) } @@ -668,14 +707,13 @@ func (em *ErrorManager) ReplaceConflictKeys( var mustKeepKvPairs *kv.Pairs for dataKvRows.Next() { - var rawKey, rawValue, rawHandle []byte - if err := dataKvRows.Scan(&rawKey, &rawValue, &rawHandle); err != nil { + var rawKey, rawValue []byte + if err := dataKvRows.Scan(&rawKey, &rawValue); err != nil { return errors.Trace(err) } - em.logger.Debug("got group raw_key, raw_value, raw_handle from table", + em.logger.Debug("got group raw_key, raw_value from table", logutil.Key("raw_key", rawKey), - zap.Binary("raw_value", rawValue), - zap.Binary("raw_handle", rawHandle)) + zap.Binary("raw_value", rawValue)) if !bytes.Equal(rawKey, previousRawKey) { previousRawKey = rawKey @@ -694,6 +732,10 @@ func (em *ErrorManager) ReplaceConflictKeys( if err != nil { return errors.Trace(err) } + if !tbl.Meta().HasClusteredIndex() { + // for nonclustered PK, need to append handle to decodedData for AddRecord + decodedData = append(decodedData, types.NewIntDatum(handle.IntValue())) + } _, err = encoder.Table.AddRecord(encoder.SessionCtx, decodedData) if err != nil { return errors.Trace(err) @@ -710,7 +752,7 @@ func (em *ErrorManager) ReplaceConflictKeys( continue } - handle, err := tablecodec.DecodeRowKey(rawHandle) + handle, err := tablecodec.DecodeRowKey(rawKey) if err != nil { return errors.Trace(err) } @@ -719,6 +761,10 @@ func (em *ErrorManager) ReplaceConflictKeys( if err != nil { return errors.Trace(err) } + if !tbl.Meta().HasClusteredIndex() { + // for nonclustered PK, need to append handle to decodedData for AddRecord + decodedData = append(decodedData, types.NewIntDatum(handle.IntValue())) + } _, err = encoder.Table.AddRecord(encoder.SessionCtx, decodedData) if err != nil { return errors.Trace(err) @@ -731,7 +777,7 @@ func (em *ErrorManager) ReplaceConflictKeys( logutil.Key("key", kvPair.Key), zap.Binary("value", kvPair.Val)) kvLatestValue, err := fnGetLatest(gCtx, kvPair.Key) - if tikverr.IsErrNotFound(err) || kvLatestValue == nil { + if tikverr.IsErrNotFound(err) { continue } if err != nil { @@ -746,11 +792,13 @@ func (em *ErrorManager) ReplaceConflictKeys( // if the KV pair is contained in mustKeepKvPairs, we cannot delete it // if not, delete the KV pair - isContained := slices.ContainsFunc(mustKeepKvPairs.Pairs, func(mustKeepKvPair common.KvPair) bool { - return bytes.Equal(mustKeepKvPair.Key, kvPair.Key) && bytes.Equal(mustKeepKvPair.Val, kvPair.Val) - }) - if isContained { - continue + if mustKeepKvPairs != nil { + isContained := slices.ContainsFunc(mustKeepKvPairs.Pairs, func(mustKeepKvPair common.KvPair) bool { + return bytes.Equal(mustKeepKvPair.Key, kvPair.Key) && bytes.Equal(mustKeepKvPair.Val, kvPair.Val) + }) + if isContained { + continue + } } if err := fnDeleteKey(gCtx, kvPair.Key); err != nil { diff --git a/br/pkg/lightning/errormanager/errormanager_test.go b/br/pkg/lightning/errormanager/errormanager_test.go index 83eeaa006866d..0cfa3345de1bc 100644 --- a/br/pkg/lightning/errormanager/errormanager_test.go +++ b/br/pkg/lightning/errormanager/errormanager_test.go @@ -67,7 +67,7 @@ func TestInit(t *testing.T) { em.conflictV1Enabled = true mock.ExpectExec("CREATE SCHEMA IF NOT EXISTS `lightning_errors`;"). WillReturnResult(sqlmock.NewResult(1, 1)) - mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v1.*"). + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v2.*"). WillReturnResult(sqlmock.NewResult(2, 1)) err = em.Init(ctx) require.NoError(t, err) @@ -79,7 +79,7 @@ func TestInit(t *testing.T) { WillReturnResult(sqlmock.NewResult(5, 1)) mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.type_error_v1.*"). WillReturnResult(sqlmock.NewResult(6, 1)) - mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v1.*"). + mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_error_v2.*"). WillReturnResult(sqlmock.NewResult(7, 1)) mock.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_errors`\\.conflict_records.*"). WillReturnResult(sqlmock.NewResult(7, 1)) @@ -186,7 +186,7 @@ func TestRemoveAllConflictKeys(t *testing.T) { require.Equal(t, totalRows, resolved.Load()) } -func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { +func TestReplaceConflictOneKey(t *testing.T) { column1 := &model.ColumnInfo{ ID: 1, Name: model.NewCIStr("a"), @@ -207,7 +207,6 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { Hidden: true, State: model.StatePublic, } - column2.AddFlag(mysql.UniqueKeyFlag) column3 := &model.ColumnInfo{ ID: 3, @@ -221,7 +220,7 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { index := &model.IndexInfo{ ID: 1, - Name: model.NewCIStr("uni_b"), + Name: model.NewCIStr("key_b"), Table: model.NewCIStr(""), Columns: []*model.IndexColumn{ { @@ -229,13 +228,13 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { Offset: 1, Length: -1, }}, - Unique: true, + Unique: false, Primary: false, State: model.StatePublic, } table := &model.TableInfo{ - ID: 75, + ID: 104, Name: model.NewCIStr("a"), Charset: "utf8mb4", Collate: "utf8mb4_bin", @@ -271,18 +270,38 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { types.NewIntDatum(6), types.NewStringDatum("2.csv"), } + data3 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(3), + types.NewStringDatum("3.csv"), + } + data4 := []types.Datum{ + types.NewIntDatum(3), + types.NewIntDatum(4), + types.NewStringDatum("4.csv"), + } + data5 := []types.Datum{ + types.NewIntDatum(5), + types.NewIntDatum(4), + types.NewStringDatum("5.csv"), + } _, 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) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data4) + require.NoError(t, err) + _, err = encoder.Table.AddRecord(encoder.SessionCtx, data5) + require.NoError(t, err) kvPairs := encoder.SessionCtx.TakeKvPairs() - data1IndexKey := kvPairs.Pairs[1].Key - data1IndexValue := kvPairs.Pairs[1].Val - data2IndexValue := kvPairs.Pairs[3].Val - data1RowKey := kvPairs.Pairs[0].Key - data2RowKey := kvPairs.Pairs[2].Key - data2RowValue := kvPairs.Pairs[2].Val + data1IndexKey := kvPairs.Pairs[7].Key + data1IndexValue := kvPairs.Pairs[7].Val + data1RowKey := kvPairs.Pairs[4].Key + data1RowValue := kvPairs.Pairs[4].Val + data2RowValue := kvPairs.Pairs[6].Val db, mockDB, err := sqlmock.New() require.NoError(t, err) @@ -295,14 +314,14 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { 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_v1.*"). + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). WillReturnResult(sqlmock.NewResult(2, 1)) - mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v1 WHERE table_name = ? AND index_name <> 'PRIMARY' ORDER BY raw_key\\E"). - WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). - AddRow(data1IndexKey, "uni_b", data1IndexValue, data1RowKey). - AddRow(data1IndexKey, "uni_b", data2IndexValue, data2RowKey)) - mockDB.ExpectQuery("\\QSELECT raw_key, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v1 WHERE table_name = ? AND index_name = 'PRIMARY' ORDER BY raw_key\\E"). - WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value", "raw_handle"})) + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). + WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"})) + 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(data1RowKey, data1RowValue). + AddRow(data1RowKey, data2RowValue)) cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -321,15 +340,15 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { switch { case bytes.Equal(key, data1IndexKey): return data1IndexValue, nil - case bytes.Equal(key, data2RowKey): - return data2RowValue, nil + case bytes.Equal(key, data1RowKey): + return data1RowValue, nil default: return nil, fmt.Errorf("key %v is not expected", key) } }, func(ctx context.Context, key []byte) error { fnDeleteKeyCount.Add(1) - if !bytes.Equal(key, data2RowKey) { + if !bytes.Equal(key, data1IndexKey) { return fmt.Errorf("key %v is not expected", key) } return nil @@ -342,7 +361,7 @@ func TestReplaceConflictKeysIndexKvChecking(t *testing.T) { require.NoError(t, err) } -func TestReplaceConflictKeys(t *testing.T) { +func TestReplaceConflictOneUniqueKey(t *testing.T) { column1 := &model.ColumnInfo{ ID: 1, Name: model.NewCIStr("a"), @@ -456,7 +475,7 @@ func TestReplaceConflictKeys(t *testing.T) { data1IndexKey := kvPairs.Pairs[7].Key data3IndexKey := kvPairs.Pairs[1].Key - data1IndexValue := kvPairs.Pairs[5].Val + data1IndexValue := kvPairs.Pairs[7].Val data2IndexValue := kvPairs.Pairs[9].Val data3IndexValue := kvPairs.Pairs[1].Val data4IndexValue := kvPairs.Pairs[3].Val @@ -480,18 +499,28 @@ func TestReplaceConflictKeys(t *testing.T) { 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_v1.*"). + mockDB.ExpectExec("CREATE TABLE IF NOT EXISTS `lightning_task_info`\\.conflict_error_v2.*"). WillReturnResult(sqlmock.NewResult(2, 1)) - mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v1 WHERE table_name = ? AND index_name <> 'PRIMARY' ORDER BY raw_key\\E"). + mockDB.ExpectQuery("\\QSELECT raw_key, index_name, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v2 WHERE table_name = ? AND is_data_kv = 0 ORDER BY raw_key\\E"). WillReturnRows(sqlmock.NewRows([]string{"raw_key", "index_name", "raw_value", "raw_handle"}). AddRow(data1IndexKey, "uni_b", data1IndexValue, data1RowKey). AddRow(data1IndexKey, "uni_b", data2IndexValue, data2RowKey). AddRow(data3IndexKey, "uni_b", data3IndexValue, data3RowKey). AddRow(data3IndexKey, "uni_b", data4IndexValue, data4RowKey)) - mockDB.ExpectQuery("\\QSELECT raw_key, raw_value, raw_handle FROM `lightning_task_info`.conflict_error_v1 WHERE table_name = ? AND index_name = 'PRIMARY' ORDER BY raw_key\\E"). - WillReturnRows(sqlmock.NewRows([]string{"raw_key", "raw_value", "raw_handle"}). - AddRow(data1RowKey, data1RowValue, data1RowKey). - AddRow(data1RowKey, data3RowValue, data1RowKey)) + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "test", nil, nil, data2RowKey, data2RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + mockDB.ExpectBegin() + mockDB.ExpectExec("INSERT INTO `lightning_task_info`\\.conflict_error_v2.*"). + WithArgs(0, "test", nil, nil, data4RowKey, data4RowValue, 1). + WillReturnResult(driver.ResultNoRows) + mockDB.ExpectCommit() + 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(data1RowKey, data1RowValue). + AddRow(data1RowKey, data3RowValue)) cfg := config.NewConfig() cfg.TikvImporter.DuplicateResolution = config.DupeResAlgReplace @@ -641,7 +670,7 @@ func TestErrorMgrErrorOutput(t *testing.T) { "|\x1b[31m 1 \x1b[0m|\x1b[31m Data Type \x1b[0m|\x1b[31m 100 \x1b[0m|\x1b[31m `error_info`.`type_error_v1` \x1b[0m|\n" + "|\x1b[31m 2 \x1b[0m|\x1b[31m Data Syntax \x1b[0m|\x1b[31m 100 \x1b[0m|\x1b[31m `error_info`.`syntax_error_v1` \x1b[0m|\n" + "|\x1b[31m 3 \x1b[0m|\x1b[31m Charset Error \x1b[0m|\x1b[31m 100 \x1b[0m|\x1b[31m \x1b[0m|\n" + - "|\x1b[31m 4 \x1b[0m|\x1b[31m Unique Key Conflict \x1b[0m|\x1b[31m 100 \x1b[0m|\x1b[31m `error_info`.`conflict_error_v1` \x1b[0m|\n" + + "|\x1b[31m 4 \x1b[0m|\x1b[31m Unique Key Conflict \x1b[0m|\x1b[31m 100 \x1b[0m|\x1b[31m `error_info`.`conflict_error_v2` \x1b[0m|\n" + "+---+---------------------+-------------+----------------------------------+\n" require.Equal(t, expected, output) diff --git a/br/tests/lightning_config_max_error/run.sh b/br/tests/lightning_config_max_error/run.sh index 543971140c015..74de7022007ce 100755 --- a/br/tests/lightning_config_max_error/run.sh +++ b/br/tests/lightning_config_max_error/run.sh @@ -27,7 +27,7 @@ uniq_row_count=$( sed '1d' "${data_file}" | awk -F, '{print $1}' | sort | uniq - duplicated_row_count=$(( ${total_row_count} - ${uniq_row_count} )) run_sql 'DROP TABLE IF EXISTS mytest.testtbl' -run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v1' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' stderr_file="/tmp/${TEST_NAME}.stderr" @@ -45,7 +45,7 @@ EOF cat "${stderr_file}" grep -q "${err_msg}" "${stderr_file}" -run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v1' +run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v2' # Although conflict error number exceeds the max-error limit, # all the conflict errors are recorded, # because recording of conflict errors are executed batch by batch (batch size 1024), @@ -55,11 +55,11 @@ check_contains "COUNT(*): ${duplicated_row_count}" # import a second time run_sql 'DROP TABLE IF EXISTS mytest.testtbl' -run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v1' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' run_lightning --backend local --config "${mydir}/normal_config.toml" -run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v1' +run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v2' check_contains "COUNT(*): ${duplicated_row_count}" # Check remaining records in the target table @@ -69,11 +69,11 @@ check_contains "COUNT(*): ${uniq_row_count}" # import a third time run_sql 'DROP TABLE IF EXISTS mytest.testtbl' -run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v1' +run_sql 'DROP TABLE IF EXISTS lightning_task_info.conflict_error_v2' run_lightning --backend local --config "${mydir}/normal_config_old_style.toml" -run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v1' +run_sql 'SELECT COUNT(*) FROM lightning_task_info.conflict_error_v2' check_contains "COUNT(*): ${duplicated_row_count}" # Check remaining records in the target table diff --git a/br/tests/lightning_duplicate_detection/run.sh b/br/tests/lightning_duplicate_detection/run.sh index 6b52968387263..bba3107a31f3f 100644 --- a/br/tests/lightning_duplicate_detection/run.sh +++ b/br/tests/lightning_duplicate_detection/run.sh @@ -53,7 +53,7 @@ verify_detected_rows() { done done mapfile -t expect_rows < <(for row in "${expect_rows[@]}"; do echo "$row"; done | sort | uniq) - mapfile -t actual_rows < <(run_sql "SELECT row_data FROM lightning_task_info.conflict_error_v1 WHERE table_name = \"\`dup_detect\`.\`${table}\`\"" | + mapfile -t actual_rows < <(run_sql "SELECT row_data FROM lightning_task_info.conflict_error_v2 WHERE table_name = \"\`dup_detect\`.\`${table}\`\"" | grep "row_data:" | sed 's/^.*(//' | sed 's/).*$//' | sed 's/"//g' | sed 's/, */,/g' | sort | uniq) equal=0 if [ "${#actual_rows[@]}" = "${#expect_rows[@]}" ]; then diff --git a/br/tests/lightning_duplicate_resolution_replace/config.toml b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/config.toml similarity index 100% rename from br/tests/lightning_duplicate_resolution_replace/config.toml rename to br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/config.toml diff --git a/br/tests/lightning_duplicate_resolution_replace/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve-schema-create.sql similarity index 100% rename from br/tests/lightning_duplicate_resolution_replace/data/dup_resolve-schema-create.sql rename to br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve-schema-create.sql diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..c86cf23fe93c5 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,8 @@ +create table a ( + a int primary key clustered, + b int not null, + c int not null, + d text, + key key_b(b), + key key_c(c) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..8d5ed8ea7d518 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,8 @@ +a,b,c,d +1,1,1,1.csv +1,1,2,2.csv +2,2,3,3.csv +3,3,3,3.csv +3,3,3,4.csv +4,4,4,4.csv +5,4,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/run.sh new file mode 100644 index 0000000000000..0806f7960a21c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_clustered_pk/run.sh @@ -0,0 +1,49 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 5' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1' +check_contains 'd: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 3' +check_contains 'b: 3' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 4' +check_contains 'b: 4' +check_contains 'c: 4' +check_contains 'd: 4.csv' +check_contains 'a: 5' +check_contains 'b: 4' +check_contains 'c: 5' +check_contains 'd: 5.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/config.toml new file mode 100644 index 0000000000000..d49b2583e944c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 1 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..b1f757fb7e878 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,8 @@ +create table a ( + a int primary key nonclustered, + b int not null, + c int not null, + d text, + key key_b(b), + key key_c(c) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..8d5ed8ea7d518 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,8 @@ +a,b,c,d +1,1,1,1.csv +1,1,2,2.csv +2,2,3,3.csv +3,3,3,3.csv +3,3,3,4.csv +4,4,4,4.csv +5,4,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/run.sh new file mode 100644 index 0000000000000..0806f7960a21c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_keys_nonclustered_pk/run.sh @@ -0,0 +1,49 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 5' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1' +check_contains 'd: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 3' +check_contains 'b: 3' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 4' +check_contains 'b: 4' +check_contains 'c: 4' +check_contains 'd: 4.csv' +check_contains 'a: 5' +check_contains 'b: 4' +check_contains 'c: 5' +check_contains 'd: 5.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/config.toml new file mode 100644 index 0000000000000..d49b2583e944c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 1 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..33259ec920850 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,8 @@ +create table a ( + a int primary key clustered, + b int not null, + c int not null, + d text, + unique key uni_b(b), + unique key uni_c(c) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..8d5ed8ea7d518 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,8 @@ +a,b,c,d +1,1,1,1.csv +1,1,2,2.csv +2,2,3,3.csv +3,3,3,3.csv +3,3,3,4.csv +4,4,4,4.csv +5,4,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/run.sh new file mode 100644 index 0000000000000..edc75173cffa7 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_clustered_pk/run.sh @@ -0,0 +1,41 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 3' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1' +check_contains 'd: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 4' +check_contains 'b: 4' +check_contains 'c: 4' +check_contains 'd: 4.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/config.toml new file mode 100644 index 0000000000000..d49b2583e944c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 1 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..82a7aa56536f1 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,8 @@ +create table a ( + a int primary key nonclustered, + b int not null, + c int not null, + d text, + unique key uni_b(b), + unique key uni_c(c) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..8d5ed8ea7d518 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,8 @@ +a,b,c,d +1,1,1,1.csv +1,1,2,2.csv +2,2,3,3.csv +3,3,3,3.csv +3,3,3,4.csv +4,4,4,4.csv +5,4,5,5.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/run.sh new file mode 100644 index 0000000000000..edc75173cffa7 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_multiple_unique_keys_nonclustered_pk/run.sh @@ -0,0 +1,41 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 3' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1' +check_contains 'd: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 3' +check_contains 'd: 3.csv' +check_contains 'a: 4' +check_contains 'b: 4' +check_contains 'c: 4' +check_contains 'd: 4.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_key/config.toml new file mode 100644 index 0000000000000..1617a992eb0d0 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +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_replace_one_key/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..113efc3929c48 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key/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_replace/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve.a.1.csv similarity index 100% rename from br/tests/lightning_duplicate_resolution_replace/data/dup_resolve.a.1.csv rename to br/tests/lightning_duplicate_resolution_replace_one_key/data/dup_resolve.a.1.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_key/run.sh new file mode 100644 index 0000000000000..aa64170975f66 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key/run.sh @@ -0,0 +1,41 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 4' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 6' +check_contains 'c: 1.csv' +check_contains 'a: 2' +check_contains 'b: 6' +check_contains 'c: 2.csv' +check_contains 'a: 3' +check_contains 'b: 3' +check_contains 'c: 3.csv' +check_contains 'a: 5' +check_contains 'b: 4' +check_contains 'c: 5.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/config.toml new file mode 100644 index 0000000000000..babdb61ffe1ae --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 3 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..a483649e53cae --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key clustered, + b int not null, + c text, + key key_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..87dba4104fb2b --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,16 @@ +a,b,c +1,1,1.csv +1,1,2.csv +1,1,3.csv +2,2,1.csv +2,1,1.csv +2,2,2.csv +3,1,3.csv +3,4,4.csv +3,5,5.csv +4,3,4.csv +4,4,4.csv +4,5,4.csv +5,5,5.csv +5,6,6.csv +6,6,6.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/run.sh new file mode 100644 index 0000000000000..ea423c618e31f --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_clustered_pk/run.sh @@ -0,0 +1,47 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 6' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 1.csv' +check_contains 'a: 3' +check_contains 'b: 1' +check_contains 'c: 3.csv' +check_contains 'a: 4' +check_contains 'b: 3' +check_contains 'c: 4.csv' +check_contains 'a: 5' +check_contains 'b: 6' +check_contains 'c: 6.csv' +check_contains 'a: 6' +check_contains 'b: 6' +check_contains 'c: 6.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/config.toml new file mode 100644 index 0000000000000..babdb61ffe1ae --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 3 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..2e62887f6f1f8 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key nonclustered, + b int not null, + c text, + key key_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..87dba4104fb2b --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,16 @@ +a,b,c +1,1,1.csv +1,1,2.csv +1,1,3.csv +2,2,1.csv +2,1,1.csv +2,2,2.csv +3,1,3.csv +3,4,4.csv +3,5,5.csv +4,3,4.csv +4,4,4.csv +4,5,4.csv +5,5,5.csv +5,6,6.csv +6,6,6.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/run.sh new file mode 100644 index 0000000000000..ea423c618e31f --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_key_multiple_conflicts_nonclustered_pk/run.sh @@ -0,0 +1,47 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 6' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 1.csv' +check_contains 'a: 3' +check_contains 'b: 1' +check_contains 'c: 3.csv' +check_contains 'a: 4' +check_contains 'b: 3' +check_contains 'c: 4.csv' +check_contains 'a: 5' +check_contains 'b: 6' +check_contains 'c: 6.csv' +check_contains 'a: 6' +check_contains 'b: 6' +check_contains 'c: 6.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/config.toml new file mode 100644 index 0000000000000..d49b2583e944c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 1 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve.a-schema.sql similarity index 100% rename from br/tests/lightning_duplicate_resolution_replace/data/dup_resolve.a-schema.sql rename to br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve.a-schema.sql diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..099a240a110ee --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/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_replace/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/run.sh similarity index 100% rename from br/tests/lightning_duplicate_resolution_replace/run.sh rename to br/tests/lightning_duplicate_resolution_replace_one_unique_key_clustered_pk/run.sh diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/config.toml new file mode 100644 index 0000000000000..92ff01703a15b --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/config.toml @@ -0,0 +1,16 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 3 + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..a55ebf84d8db4 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key clustered, + b int not null, + c text, + unique key uni_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..f3b328c3c39ec --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,7 @@ +a,b,c +1,1,1.csv +1,1,2.csv +1,1,3.csv +2,2,1.csv +2,1,1.csv +2,2,2.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.2.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.2.csv new file mode 100644 index 0000000000000..f5b1be6bac6e4 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/data/dup_resolve.a.2.csv @@ -0,0 +1,9 @@ +a,b,c +3,3,3.csv +3,4,4.csv +4,3,4.csv +4,4,4.csv +4,5,4.csv +5,5,5.csv +5,6,6.csv +6,6,6.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/run.sh new file mode 100644 index 0000000000000..aa229d50b1145 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_clustered_pk/run.sh @@ -0,0 +1,38 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 3' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 1.csv' +check_contains 'a: 3' +check_contains 'b: 3' +check_contains 'c: 3.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/config.toml new file mode 100644 index 0000000000000..92ff01703a15b --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/config.toml @@ -0,0 +1,16 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 3 + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..a55ebf84d8db4 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a int primary key clustered, + b int not null, + c text, + unique key uni_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..f3b328c3c39ec --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,7 @@ +a,b,c +1,1,1.csv +1,1,2.csv +1,1,3.csv +2,2,1.csv +2,1,1.csv +2,2,2.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.2.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.2.csv new file mode 100644 index 0000000000000..f5b1be6bac6e4 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/data/dup_resolve.a.2.csv @@ -0,0 +1,9 @@ +a,b,c +3,3,3.csv +3,4,4.csv +4,3,4.csv +4,4,4.csv +4,5,4.csv +5,5,5.csv +5,6,6.csv +6,6,6.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/run.sh new file mode 100644 index 0000000000000..aa229d50b1145 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_multiple_conflicts_nonclustered_pk/run.sh @@ -0,0 +1,38 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 3' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: 1' +check_contains 'b: 1' +check_contains 'c: 1.csv' +check_contains 'a: 2' +check_contains 'b: 2' +check_contains 'c: 1.csv' +check_contains 'a: 3' +check_contains 'b: 3' +check_contains 'c: 3.csv' diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/config.toml b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/config.toml new file mode 100644 index 0000000000000..d49b2583e944c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/config.toml @@ -0,0 +1,17 @@ +[lightning] +task-info-schema-name = 'lightning_task_info' + +[tikv-importer] +backend = 'local' +duplicate-resolution = 'replace' +add-index-by-sql = false + +[checkpoint] +enable = false + +[mydumper] +batch-size = 1 +# ensure each file is its own engine to facilitate cross-engine detection. + +[mydumper.csv] +header = true diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve-schema-create.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve-schema-create.sql new file mode 100644 index 0000000000000..f8d42367a3d4c --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve-schema-create.sql @@ -0,0 +1 @@ +create schema dup_resolve; diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a-schema.sql b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a-schema.sql new file mode 100644 index 0000000000000..6b87b423cf3fa --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a-schema.sql @@ -0,0 +1,6 @@ +create table a ( + a varchar(20) primary key nonclustered, + b int not null, + c text, + unique key uni_b(b) +); diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a.1.csv b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a.1.csv new file mode 100644 index 0000000000000..f524fee5ba095 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/data/dup_resolve.a.1.csv @@ -0,0 +1,6 @@ +a,b,c +"x",6,1.csv +"y",6,2.csv +"z",3,3.csv +"z",4,4.csv +"t",4,5.csv diff --git a/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/run.sh b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/run.sh new file mode 100644 index 0000000000000..17b277e850051 --- /dev/null +++ b/br/tests/lightning_duplicate_resolution_replace_one_unique_key_nonclustered_varchar_pk/run.sh @@ -0,0 +1,35 @@ +#!/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 + +run_lightning + +# Ensure all tables are consistent. +run_sql 'admin check table dup_resolve.a' + +run_sql 'select count(*) from dup_resolve.a' +check_contains 'count(*): 2' + +run_sql 'select * from dup_resolve.a' +check_contains 'a: x' +check_contains 'b: 6' +check_contains 'c: 1.csv' +check_contains 'a: z' +check_contains 'b: 3' +check_contains 'c: 3.csv' diff --git a/br/tests/lightning_issue_40657/run.sh b/br/tests/lightning_issue_40657/run.sh index efd912017c891..a5ab8e88298cc 100644 --- a/br/tests/lightning_issue_40657/run.sh +++ b/br/tests/lightning_issue_40657/run.sh @@ -24,7 +24,7 @@ run_lightning -d "$CUR/data1" run_sql 'admin check table test.t' run_sql 'select count(*) from test.t' check_contains 'count(*): 3' -run_sql 'select count(*) from lightning_task_info.conflict_error_v1' +run_sql 'select count(*) from lightning_task_info.conflict_error_v2' check_contains 'count(*): 2' run_sql 'truncate table test.t' diff --git a/br/tests/run_group.sh b/br/tests/run_group.sh index 8da15cab19a30..3ae324cb61039 100755 --- a/br/tests/run_group.sh +++ b/br/tests/run_group.sh @@ -30,12 +30,12 @@ groups=( ["G07"]='br_clustered_index br_crypter br_table_partition br_tidb_placement_policy br_tiflash br_tikv_outage' ["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint lightning_add_index lightning_alter_random lightning_auto_columns' ["G09"]='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' - ["G10"]='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' - ["G11"]='lightning_compress lightning_concurrent-restore 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' - ["G12"]='lightning_drop_other_tables_halfway lightning_duplicate_detection lightning_duplicate_detection_new lightning_duplicate_resolution_incremental lightning_duplicate_resolution_remove lightning_duplicate_resolution_replace lightning_error_summary lightning_examples lightning_exotic_filenames lightning_extend_routes lightning_fail_fast' - ["G13"]='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' - ["G14"]='lightning_issue_40657 lightning_issue_410 lightning_issue_519 lightning_local_backend lightning_max_incr lightning_max_random lightning_multi_valued_index lightning_new_collation lightning_no_schema' - ["G15"]='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' + ["G10"]='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' + ["G11"]='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' + ["G12"]='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' + ["G13"]='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' + ["G14"]='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' + ["G15"]='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' ["G16"]='lightning_shard_rowid lightning_source_linkfile lightning_sqlmode lightning_tidb_duplicate_data lightning_tidb_rowid lightning_tiflash lightning_tikv_multi_rocksdb lightning_too_many_columns lightning_tool_135' ["G17"]='lightning_tool_1420 lightning_tool_1472 lightning_tool_241 lightning_ttl lightning_unused_config_keys lightning_various_types lightning_view lightning_write_batch lightning_write_limit lightning_pd_leader_switch' ) diff --git a/br/tidb-lightning.toml b/br/tidb-lightning.toml index 7607ff89a9547..3b780877d453a 100644 --- a/br/tidb-lightning.toml +++ b/br/tidb-lightning.toml @@ -103,7 +103,7 @@ addr = "127.0.0.1:8287" # Current supports three resolution algorithms: # - none: doesn't detect duplicate records, which has the best performance of the three algorithms, but probably leads to # inconsistent data in the target TiDB. -# - record: only records duplicate records to `lightning_task_info.conflict_error_v1` table on the target TiDB. Note that this +# - record: only records duplicate records to `lightning_task_info.conflict_error_v2` table on the target TiDB. Note that this # required the version of target TiKV version is no less than v5.2.0, otherwise it will fallback to 'none'. # - remove: records all duplicate records like the 'record' algorithm and remove all duplicate records to ensure a consistent # state in the target TiDB.