From 62c7c5e2a31a76c162abfb1555f228373081aa9b Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 21 Apr 2022 13:40:49 +0800 Subject: [PATCH 01/11] fix: auto_increment exceeds limit --- br/pkg/lightning/restore/table_restore.go | 2 +- br/pkg/lightning/restore/tidb.go | 9 +++++++-- br/pkg/lightning/restore/tidb_test.go | 4 ++++ 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 27410428ede8a..41d96a8bf76e1 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -686,7 +686,7 @@ func (tr *TableRestore) postProcess( err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoRandomType).Base()+1) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle - err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.RowIDAllocType).Base()+1) + err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.RowIDAllocType).Base())+1) } rc.alterTableLock.Unlock() saveCpErr := rc.saveStatusCheckpoint(ctx, tr.tableName, checkpoints.WholeTableEngineID, err, checkpoints.CheckpointStatusAlteredAutoInc) diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index e530f8be604a4..4e243341e3aee 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -18,6 +18,7 @@ import ( "context" "database/sql" "fmt" + "math" "strconv" "strings" @@ -373,8 +374,12 @@ func ObtainNewCollationEnabled(ctx context.Context, g glue.SQLExecutor) (bool, e // NOTE: since tidb can make sure the auto id is always be rebase even if the `incr` value is smaller // the the auto incremanet base in tidb side, we needn't fetch currently auto increment value here. // See: https://github.com/pingcap/tidb/blob/64698ef9a3358bfd0fdc323996bb7928a56cadca/ddl/ddl_api.go#L2528-L2533 -func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName string, incr int64) error { - logger := log.With(zap.String("table", tableName), zap.Int64("auto_increment", incr)) +func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName string, incr uint64) error { + logger := log.With(zap.String("table", tableName), zap.Uint64("auto_increment", incr)) + if incr > math.MaxInt64 { + logger.Warn("auto_increment out of the maximum value TiDB supports", zap.Uint64("auto_increment", incr)) + return nil + } query := fmt.Sprintf("ALTER TABLE %s AUTO_INCREMENT=%d", tableName, incr) task := logger.Begin(zap.InfoLevel, "alter table auto_increment") err := g.ExecuteWithLog(ctx, query, "alter table auto_increment", logger) diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index db5ccc8b7a845..c57c362561284 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -17,6 +17,7 @@ package restore import ( "context" "database/sql" + "math" "testing" "github.com/DATA-DOG/go-sqlmock" @@ -409,6 +410,9 @@ func TestAlterAutoInc(t *testing.T) { err := AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) require.NoError(t, err) + + err = AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64+1)) + require.NoError(t, err) } func TestAlterAutoRandom(t *testing.T) { From 061987bd287362f3cbdaa04938737dc0d4f0c1ba Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 21 Apr 2022 19:20:52 +0800 Subject: [PATCH 02/11] fix comment: fix auto_random & add it --- br/pkg/lightning/restore/table_restore.go | 2 +- br/pkg/lightning/restore/tidb.go | 24 ++++++++-- br/pkg/lightning/restore/tidb_test.go | 8 +++- br/tests/lightning_max_incr/config.toml | 2 + .../data/db-schema-create.sql | 1 + .../data/db.test-schema.sql | 5 ++ .../data/db.test.000000000.csv | 3 ++ .../data/db.test1-schema.sql | 5 ++ .../data/db.test1.000000000.csv | 3 ++ br/tests/lightning_max_incr/run.sh | 47 +++++++++++++++++++ 10 files changed, 93 insertions(+), 7 deletions(-) create mode 100644 br/tests/lightning_max_incr/config.toml create mode 100644 br/tests/lightning_max_incr/data/db-schema-create.sql create mode 100644 br/tests/lightning_max_incr/data/db.test-schema.sql create mode 100644 br/tests/lightning_max_incr/data/db.test.000000000.csv create mode 100644 br/tests/lightning_max_incr/data/db.test1-schema.sql create mode 100644 br/tests/lightning_max_incr/data/db.test1.000000000.csv create mode 100644 br/tests/lightning_max_incr/run.sh diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 41d96a8bf76e1..08a0c88d2507e 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -683,7 +683,7 @@ func (tr *TableRestore) postProcess( tblInfo := tr.tableInfo.Core var err error if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() { - err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, tr.alloc.Get(autoid.AutoRandomType).Base()+1) + err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.RowIDAllocType).Base())+1) diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index 4e243341e3aee..fe3b49c56f86f 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.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" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -375,12 +376,16 @@ func ObtainNewCollationEnabled(ctx context.Context, g glue.SQLExecutor) (bool, e // the the auto incremanet base in tidb side, we needn't fetch currently auto increment value here. // See: https://github.com/pingcap/tidb/blob/64698ef9a3358bfd0fdc323996bb7928a56cadca/ddl/ddl_api.go#L2528-L2533 func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName string, incr uint64) error { + var query string logger := log.With(zap.String("table", tableName), zap.Uint64("auto_increment", incr)) if incr > math.MaxInt64 { - logger.Warn("auto_increment out of the maximum value TiDB supports", zap.Uint64("auto_increment", incr)) - return nil + // automatically set max value + logger.Warn("auto_increment out of the maximum value TiDB supports, automatically set to the max", zap.Uint64("auto_increment", incr)) + incr = math.MaxInt64 + query = fmt.Sprintf("ALTER TABLE %s FORCE AUTO_INCREMENT=%d", tableName, incr) + } else { + query = fmt.Sprintf("ALTER TABLE %s AUTO_INCREMENT=%d", tableName, incr) } - query := fmt.Sprintf("ALTER TABLE %s AUTO_INCREMENT=%d", tableName, incr) task := logger.Begin(zap.InfoLevel, "alter table auto_increment") err := g.ExecuteWithLog(ctx, query, "alter table auto_increment", logger) task.End(zap.ErrorLevel, err) @@ -393,8 +398,17 @@ func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName strin return errors.Annotatef(err, "%s", query) } -func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase int64) error { - logger := log.With(zap.String("table", tableName), zap.Int64("auto_random", randomBase)) +func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase uint64) error { + logger := log.With(zap.String("table", tableName), zap.Uint64("auto_random", randomBase)) + // according to https://docs.pingcap.com/zh/tidb/dev/auto-random + // auto_random_base is related to shard_bits from (0, 15] + // which can be specified by users. + // To make the random_base always valid, we conservatively set it to 15. + maxRandom := 1<<(64-autoid.MaxAutoRandomBits-1) - 1 // one bit for sign + if randomBase > uint64(maxRandom) { + logger.Warn("auto_random out of the maximum value TiDB supports, automatically set to the max", zap.Uint64("auto_random", randomBase)) + return nil + } query := fmt.Sprintf("ALTER TABLE %s AUTO_RANDOM_BASE=%d", tableName, randomBase) task := logger.Begin(zap.InfoLevel, "alter table auto_random") err := g.ExecuteWithLog(ctx, query, "alter table auto_random_base", logger) diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index c57c362561284..332f894e9ad6d 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -405,13 +405,16 @@ func TestAlterAutoInc(t *testing.T) { s.mockDB. ExpectExec("\\QALTER TABLE `db`.`table` AUTO_INCREMENT=12345\\E"). WillReturnResult(sqlmock.NewResult(1, 1)) + s.mockDB. + ExpectExec("\\QALTER TABLE `db`.`table` FORCE AUTO_INCREMENT=9223372036854775807\\E"). + WillReturnResult(sqlmock.NewResult(1, 1)) s.mockDB. ExpectClose() err := AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) require.NoError(t, err) - err = AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64+1)) + err = AlterAutoIncrement(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64)+1) require.NoError(t, err) } @@ -428,6 +431,9 @@ func TestAlterAutoRandom(t *testing.T) { err := AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) require.NoError(t, err) + + err = AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64)+1) + require.NoError(t, err) } func TestObtainRowFormatVersionSucceed(t *testing.T) { diff --git a/br/tests/lightning_max_incr/config.toml b/br/tests/lightning_max_incr/config.toml new file mode 100644 index 0000000000000..d2152b47c922a --- /dev/null +++ b/br/tests/lightning_max_incr/config.toml @@ -0,0 +1,2 @@ +[tikv-importer] +backend = 'local' diff --git a/br/tests/lightning_max_incr/data/db-schema-create.sql b/br/tests/lightning_max_incr/data/db-schema-create.sql new file mode 100644 index 0000000000000..c88b0e3150e76 --- /dev/null +++ b/br/tests/lightning_max_incr/data/db-schema-create.sql @@ -0,0 +1 @@ +create database db; \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test-schema.sql b/br/tests/lightning_max_incr/data/db.test-schema.sql new file mode 100644 index 0000000000000..494571fe9736d --- /dev/null +++ b/br/tests/lightning_max_incr/data/db.test-schema.sql @@ -0,0 +1,5 @@ +create table test( + a bigint auto_increment, + b int, + primary key(a) +); \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test.000000000.csv b/br/tests/lightning_max_incr/data/db.test.000000000.csv new file mode 100644 index 0000000000000..abe9c63c2a028 --- /dev/null +++ b/br/tests/lightning_max_incr/data/db.test.000000000.csv @@ -0,0 +1,3 @@ +"a","b" +1,2 +9223372036854775807,3 \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test1-schema.sql b/br/tests/lightning_max_incr/data/db.test1-schema.sql new file mode 100644 index 0000000000000..d7f7b3a9b941d --- /dev/null +++ b/br/tests/lightning_max_incr/data/db.test1-schema.sql @@ -0,0 +1,5 @@ +create table test1( + a bigint auto_random, + b int, + primary key(a) +); \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test1.000000000.csv b/br/tests/lightning_max_incr/data/db.test1.000000000.csv new file mode 100644 index 0000000000000..abe9c63c2a028 --- /dev/null +++ b/br/tests/lightning_max_incr/data/db.test1.000000000.csv @@ -0,0 +1,3 @@ +"a","b" +1,2 +9223372036854775807,3 \ No newline at end of file diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh new file mode 100644 index 0000000000000..7eac828d7c921 --- /dev/null +++ b/br/tests/lightning_max_incr/run.sh @@ -0,0 +1,47 @@ +#!/bin/sh +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# 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 4 0 0 'local backend' || exit 0 + +ENGINE_COUNT=6 + +check_result() { + run_sql 'SHOW DATABASES;' + check_contains 'Database: db'; + run_sql 'SHOW TABLES IN db;' + check_contains 'Tables_in_db: test' + check_contains 'Tables_in_db: test1' + run_sql 'SELECT count(*) FROM db.test;' + check_contains: 'count(*): 2' + run_sql 'SELECT count(*) FROM db.test1;' + check_contains: 'count(*): 2' +} + +cleanup() { + rm -f $TEST_DIR/lightning.log + rm -rf $TEST_DIR/sst + run_sql 'DROP DATABASE IF EXISTS db;' +} + +cleanup + +run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" +check_result +run_sql 'INSERT INTO db.test(b) VALUES(11);' +check_contains 'ERROR' +cleanup From 95a2442b2b4cc6c462a48dc2c61ec103c49bd41a Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Fri, 22 Apr 2022 13:31:33 +0800 Subject: [PATCH 03/11] fix: limit auto_random maximum --- br/pkg/lightning/restore/table_restore.go | 5 ++++- br/pkg/lightning/restore/tidb.go | 12 +++--------- br/pkg/lightning/restore/tidb_test.go | 7 +++++-- br/tests/lightning_max_incr/data/db.test1-schema.sql | 2 +- br/tests/lightning_max_incr/run.sh | 4 ++-- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 08a0c88d2507e..2e32d5e1af049 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -683,7 +683,10 @@ func (tr *TableRestore) postProcess( tblInfo := tr.tableInfo.Core var err error if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() { - err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1) + var maxAutoRandom uint64 + autoRandomBits := tblInfo.AutoRandomBits + maxAutoRandom = 1<<(64-1-autoRandomBits) - 1 // one bit for sign + err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, uint64(maxAutoRandom)) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.RowIDAllocType).Base())+1) diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index fe3b49c56f86f..cd65972a47ff5 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -30,7 +30,6 @@ 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" - "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/format" @@ -398,16 +397,11 @@ func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName strin return errors.Annotatef(err, "%s", query) } -func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase uint64) error { +func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase uint64, maxAutoRandom uint64) error { logger := log.With(zap.String("table", tableName), zap.Uint64("auto_random", randomBase)) - // according to https://docs.pingcap.com/zh/tidb/dev/auto-random - // auto_random_base is related to shard_bits from (0, 15] - // which can be specified by users. - // To make the random_base always valid, we conservatively set it to 15. - maxRandom := 1<<(64-autoid.MaxAutoRandomBits-1) - 1 // one bit for sign - if randomBase > uint64(maxRandom) { + if randomBase > maxAutoRandom { logger.Warn("auto_random out of the maximum value TiDB supports, automatically set to the max", zap.Uint64("auto_random", randomBase)) - return nil + randomBase = maxAutoRandom } query := fmt.Sprintf("ALTER TABLE %s AUTO_RANDOM_BASE=%d", tableName, randomBase) task := logger.Begin(zap.InfoLevel, "alter table auto_random") diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index 332f894e9ad6d..5e261fd88d0f5 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -426,13 +426,16 @@ func TestAlterAutoRandom(t *testing.T) { s.mockDB. ExpectExec("\\QALTER TABLE `db`.`table` AUTO_RANDOM_BASE=12345\\E"). WillReturnResult(sqlmock.NewResult(1, 1)) + s.mockDB. + ExpectExec("\\QALTER TABLE `db`.`table` AUTO_RANDOM_BASE=288230376151711743\\E"). + WillReturnResult(sqlmock.NewResult(1, 1)) s.mockDB. ExpectClose() - err := AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345) + err := AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345, 288230376151711743) require.NoError(t, err) - err = AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64)+1) + err = AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64)+1, 288230376151711743) require.NoError(t, err) } diff --git a/br/tests/lightning_max_incr/data/db.test1-schema.sql b/br/tests/lightning_max_incr/data/db.test1-schema.sql index d7f7b3a9b941d..5ab8c5e9b32ef 100644 --- a/br/tests/lightning_max_incr/data/db.test1-schema.sql +++ b/br/tests/lightning_max_incr/data/db.test1-schema.sql @@ -1,5 +1,5 @@ create table test1( - a bigint auto_random, + a bigint auto_random(10), b int, primary key(a) ); \ No newline at end of file diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 7eac828d7c921..9c1a7fc84c6dc 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -27,9 +27,9 @@ check_result() { check_contains 'Tables_in_db: test' check_contains 'Tables_in_db: test1' run_sql 'SELECT count(*) FROM db.test;' - check_contains: 'count(*): 2' + check_contains 'count(*): 2' run_sql 'SELECT count(*) FROM db.test1;' - check_contains: 'count(*): 2' + check_contains 'count(*): 2' } cleanup() { From 54b2dae3ea992ef5f2b94b39fb4ebe4bd84afb34 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Fri, 22 Apr 2022 13:53:28 +0800 Subject: [PATCH 04/11] fix: lint --- br/pkg/lightning/restore/table_restore.go | 2 +- br/tests/lightning_max_incr/run.sh | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 2e32d5e1af049..f2a840b567bbf 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -686,7 +686,7 @@ func (tr *TableRestore) postProcess( var maxAutoRandom uint64 autoRandomBits := tblInfo.AutoRandomBits maxAutoRandom = 1<<(64-1-autoRandomBits) - 1 // one bit for sign - err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, uint64(maxAutoRandom)) + err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, maxAutoRandom) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle err = AlterAutoIncrement(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.RowIDAllocType).Base())+1) diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 9c1a7fc84c6dc..1b2cac3a2458f 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -42,6 +42,7 @@ cleanup run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" check_result -run_sql 'INSERT INTO db.test(b) VALUES(11);' -check_contains 'ERROR' +run_sql 'INSERT INTO db.test(b) VALUES(11) ON DUPLICATE KEY UPDATE b=10000;' # verify incr set to MaxInt64 +run_sql 'SELECT b FROM db.test;' +check_contains 'b: 10000' cleanup From 53c8056ba560f57550fed51cb8f0d2ff4a7374ae Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Fri, 22 Apr 2022 16:41:32 +0800 Subject: [PATCH 05/11] fix: comment --- br/pkg/lightning/restore/table_restore.go | 8 +++++-- br/tests/_utils/run_sql_fail | 28 +++++++++++++++++++++++ br/tests/lightning_max_incr/run.sh | 6 ++++- 3 files changed, 39 insertions(+), 3 deletions(-) create mode 100644 br/tests/_utils/run_sql_fail diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index f2a840b567bbf..92c7fff03f33f 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -684,8 +684,12 @@ func (tr *TableRestore) postProcess( var err error if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() { var maxAutoRandom uint64 - autoRandomBits := tblInfo.AutoRandomBits - maxAutoRandom = 1<<(64-1-autoRandomBits) - 1 // one bit for sign + autoRandomBits := tblInfo.AutoRandomBits // range from (0, 15] + if !tblInfo.IsAutoRandomBitColUnsigned() { + // if auto_random is signed, leave one extra bit + autoRandomBits += 1 + } + maxAutoRandom = 1<<(64-autoRandomBits) - 1 err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, maxAutoRandom) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle diff --git a/br/tests/_utils/run_sql_fail b/br/tests/_utils/run_sql_fail new file mode 100644 index 0000000000000..18f48b0e30b43 --- /dev/null +++ b/br/tests/_utils/run_sql_fail @@ -0,0 +1,28 @@ +#!/bin/bash +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# 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 -xu + +SQL="$1" +shift + +echo "[$(date)] Executing SQL: $SQL" > "$TEST_DIR/sql_res.$TEST_NAME.txt" +mysql -uroot -h127.0.0.1 -P4000 \ + --ssl-ca="$TEST_DIR/certs/ca.pem" \ + --ssl-cert="$TEST_DIR/certs/curl.pem" \ + --ssl-key="$TEST_DIR/certs/curl.key" \ + "$@" \ + --default-character-set utf8 -E -e "$SQL" 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 1b2cac3a2458f..3020c1cb78cd0 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -42,7 +42,11 @@ cleanup run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" check_result -run_sql 'INSERT INTO db.test(b) VALUES(11) ON DUPLICATE KEY UPDATE b=10000;' # verify incr set to MaxInt64 +# local-backend set auto_increment to 9223372036854775807 after importing +run_sql_fail 'INSERT INTO db.test(b) VALUES(11);' # will not succeed +check_contains 'ERROR' +# duplicate key update +run_sql 'INSERT INTO db.test(b) VALUES(11) ON DUPLICATE KEY UPDATE b=10000;' run_sql 'SELECT b FROM db.test;' check_contains 'b: 10000' cleanup From 23ca4675e993505047be961d82dc0125187a7222 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Sun, 24 Apr 2022 12:40:23 +0800 Subject: [PATCH 06/11] revert: run_sql_fail --- br/tests/_utils/run_sql_fail | 28 ---------------------------- br/tests/lightning_max_incr/run.sh | 3 ++- 2 files changed, 2 insertions(+), 29 deletions(-) delete mode 100644 br/tests/_utils/run_sql_fail diff --git a/br/tests/_utils/run_sql_fail b/br/tests/_utils/run_sql_fail deleted file mode 100644 index 18f48b0e30b43..0000000000000 --- a/br/tests/_utils/run_sql_fail +++ /dev/null @@ -1,28 +0,0 @@ -#!/bin/bash -# -# Copyright 2022 PingCAP, Inc. -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# 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 -xu - -SQL="$1" -shift - -echo "[$(date)] Executing SQL: $SQL" > "$TEST_DIR/sql_res.$TEST_NAME.txt" -mysql -uroot -h127.0.0.1 -P4000 \ - --ssl-ca="$TEST_DIR/certs/ca.pem" \ - --ssl-cert="$TEST_DIR/certs/curl.pem" \ - --ssl-key="$TEST_DIR/certs/curl.key" \ - "$@" \ - --default-character-set utf8 -E -e "$SQL" 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 3020c1cb78cd0..ae9fc379f330b 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -43,7 +43,8 @@ cleanup run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" check_result # local-backend set auto_increment to 9223372036854775807 after importing -run_sql_fail 'INSERT INTO db.test(b) VALUES(11);' # will not succeed +# sql fail because of of duplicate key +run_sql 'INSERT INTO db.test(b) VALUES(11);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" check_contains 'ERROR' # duplicate key update run_sql 'INSERT INTO db.test(b) VALUES(11) ON DUPLICATE KEY UPDATE b=10000;' From a14195f5db1c1c207517ed4db251b9976a8cc58c Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Mon, 25 Apr 2022 10:59:04 +0800 Subject: [PATCH 07/11] update it --- br/pkg/lightning/restore/table_restore.go | 7 ++++--- br/tests/lightning_max_incr/run.sh | 4 ---- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 92c7fff03f33f..573245a3e9812 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -683,13 +683,14 @@ func (tr *TableRestore) postProcess( tblInfo := tr.tableInfo.Core var err error if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() { - var maxAutoRandom uint64 + var maxAutoRandom, autoRandomTotalBits uint64 + autoRandomTotalBits = 64 autoRandomBits := tblInfo.AutoRandomBits // range from (0, 15] if !tblInfo.IsAutoRandomBitColUnsigned() { // if auto_random is signed, leave one extra bit - autoRandomBits += 1 + autoRandomTotalBits = 63 } - maxAutoRandom = 1<<(64-autoRandomBits) - 1 + maxAutoRandom = 1<<(autoRandomTotalBits-autoRandomBits) - 1 err = AlterAutoRandom(ctx, rc.tidbGlue.GetSQLExecutor(), tr.tableName, uint64(tr.alloc.Get(autoid.AutoRandomType).Base())+1, maxAutoRandom) } else if common.TableHasAutoRowID(tblInfo) || tblInfo.GetAutoIncrementColInfo() != nil { // only alter auto increment id iff table contains auto-increment column or generated handle diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index ae9fc379f330b..f834f64373a07 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -46,8 +46,4 @@ check_result # sql fail because of of duplicate key run_sql 'INSERT INTO db.test(b) VALUES(11);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" check_contains 'ERROR' -# duplicate key update -run_sql 'INSERT INTO db.test(b) VALUES(11) ON DUPLICATE KEY UPDATE b=10000;' -run_sql 'SELECT b FROM db.test;' -check_contains 'b: 10000' cleanup From a7a11b795c828620399359304bc16efb75d60f61 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Mon, 25 Apr 2022 17:30:31 +0800 Subject: [PATCH 08/11] fix it: prove db.test1 valid --- br/tests/lightning_max_incr/run.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index f834f64373a07..8fc713c7b2c7c 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -46,4 +46,7 @@ check_result # sql fail because of of duplicate key run_sql 'INSERT INTO db.test(b) VALUES(11);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" check_contains 'ERROR' +# sql succeed: though 9223372036854775807 is definitely out-of-range of max_auto_random_base +# but lightning sets it to max_auto_random_base instead of 9223372036854775807 +run_sql 'INSERT INTO db.test1(b) VALUES(22);' cleanup From 17b9c3346d27e6fc9db1f7f7b9de0d352e6d61a5 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 28 Apr 2022 14:50:12 +0800 Subject: [PATCH 09/11] fix it: refine auto_random testcase --- br/tests/lightning_max_incr/data/db.test1.000000000.csv | 2 +- br/tests/lightning_max_incr/run.sh | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/br/tests/lightning_max_incr/data/db.test1.000000000.csv b/br/tests/lightning_max_incr/data/db.test1.000000000.csv index abe9c63c2a028..8de1e50edd09a 100644 --- a/br/tests/lightning_max_incr/data/db.test1.000000000.csv +++ b/br/tests/lightning_max_incr/data/db.test1.000000000.csv @@ -1,3 +1,3 @@ "a","b" 1,2 -9223372036854775807,3 \ No newline at end of file +9007199254740992,3 \ No newline at end of file diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 8fc713c7b2c7c..8980e408cc29c 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -46,7 +46,8 @@ check_result # sql fail because of of duplicate key run_sql 'INSERT INTO db.test(b) VALUES(11);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" check_contains 'ERROR' -# sql succeed: though 9223372036854775807 is definitely out-of-range of max_auto_random_base -# but lightning sets it to max_auto_random_base instead of 9223372036854775807 +# auto_random_max: 2^{64-1-10} - 1 = 2^53 - 1 +# sql succeed: though 9007199254740992 (2^53) is out-of-range of max_auto_random_base +# but lightning sets it to max_auto_random_base instead of 9007199254740991 run_sql 'INSERT INTO db.test1(b) VALUES(22);' cleanup From 7ec36158e6cb422eccdfa3c74a4eeb936135fc83 Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 28 Apr 2022 17:53:52 +0800 Subject: [PATCH 10/11] fix it: refactor auto_random and auto_incr --- br/pkg/lightning/restore/tidb.go | 8 ++- .../data/db.test.000000000.csv | 2 +- .../data/db.test1-schema.sql | 2 +- .../data/db.test1.000000000.csv | 2 +- br/tests/lightning_max_incr/run.sh | 13 ++-- br/tests/lightning_max_random/config.toml | 2 + .../data/db-schema-create.sql | 1 + .../data/db.test-schema.sql | 5 ++ .../data/db.test.000000000.csv | 3 + .../data/db.test1-schema.sql | 5 ++ .../data/db.test1.000000000.csv | 3 + .../data/db.test2-schema.sql | 5 ++ .../data/db.test2.000000000.csv | 3 + br/tests/lightning_max_random/run.sh | 65 +++++++++++++++++++ 14 files changed, 107 insertions(+), 12 deletions(-) create mode 100644 br/tests/lightning_max_random/config.toml create mode 100644 br/tests/lightning_max_random/data/db-schema-create.sql create mode 100644 br/tests/lightning_max_random/data/db.test-schema.sql create mode 100644 br/tests/lightning_max_random/data/db.test.000000000.csv create mode 100644 br/tests/lightning_max_random/data/db.test1-schema.sql create mode 100644 br/tests/lightning_max_random/data/db.test1.000000000.csv create mode 100644 br/tests/lightning_max_random/data/db.test2-schema.sql create mode 100644 br/tests/lightning_max_random/data/db.test2.000000000.csv create mode 100644 br/tests/lightning_max_random/run.sh diff --git a/br/pkg/lightning/restore/tidb.go b/br/pkg/lightning/restore/tidb.go index cd65972a47ff5..a6f831ef0aa14 100644 --- a/br/pkg/lightning/restore/tidb.go +++ b/br/pkg/lightning/restore/tidb.go @@ -399,9 +399,13 @@ func AlterAutoIncrement(ctx context.Context, g glue.SQLExecutor, tableName strin func AlterAutoRandom(ctx context.Context, g glue.SQLExecutor, tableName string, randomBase uint64, maxAutoRandom uint64) error { logger := log.With(zap.String("table", tableName), zap.Uint64("auto_random", randomBase)) - if randomBase > maxAutoRandom { - logger.Warn("auto_random out of the maximum value TiDB supports, automatically set to the max", zap.Uint64("auto_random", randomBase)) + if randomBase == maxAutoRandom+1 { + // insert a tuple with key maxAutoRandom randomBase = maxAutoRandom + } else if randomBase > maxAutoRandom { + // TiDB does nothing when inserting an overflow value + logger.Warn("auto_random out of the maximum value TiDB supports") + return nil } query := fmt.Sprintf("ALTER TABLE %s AUTO_RANDOM_BASE=%d", tableName, randomBase) task := logger.Begin(zap.InfoLevel, "alter table auto_random") diff --git a/br/tests/lightning_max_incr/data/db.test.000000000.csv b/br/tests/lightning_max_incr/data/db.test.000000000.csv index abe9c63c2a028..cb1603f9d4a71 100644 --- a/br/tests/lightning_max_incr/data/db.test.000000000.csv +++ b/br/tests/lightning_max_incr/data/db.test.000000000.csv @@ -1,3 +1,3 @@ "a","b" 1,2 -9223372036854775807,3 \ No newline at end of file +9223372036854775805,3 \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test1-schema.sql b/br/tests/lightning_max_incr/data/db.test1-schema.sql index 5ab8c5e9b32ef..d44068487af8b 100644 --- a/br/tests/lightning_max_incr/data/db.test1-schema.sql +++ b/br/tests/lightning_max_incr/data/db.test1-schema.sql @@ -1,5 +1,5 @@ create table test1( - a bigint auto_random(10), + a bigint auto_increment, b int, primary key(a) ); \ No newline at end of file diff --git a/br/tests/lightning_max_incr/data/db.test1.000000000.csv b/br/tests/lightning_max_incr/data/db.test1.000000000.csv index 8de1e50edd09a..abe9c63c2a028 100644 --- a/br/tests/lightning_max_incr/data/db.test1.000000000.csv +++ b/br/tests/lightning_max_incr/data/db.test1.000000000.csv @@ -1,3 +1,3 @@ "a","b" 1,2 -9007199254740992,3 \ No newline at end of file +9223372036854775807,3 \ No newline at end of file diff --git a/br/tests/lightning_max_incr/run.sh b/br/tests/lightning_max_incr/run.sh index 8980e408cc29c..ce044c0230623 100644 --- a/br/tests/lightning_max_incr/run.sh +++ b/br/tests/lightning_max_incr/run.sh @@ -40,14 +40,13 @@ cleanup() { cleanup +# db.test contains key that is less than int64 - 1 +# while db.test1 contains key that equals int64 - 1 run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" check_result -# local-backend set auto_increment to 9223372036854775807 after importing -# sql fail because of of duplicate key -run_sql 'INSERT INTO db.test(b) VALUES(11);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" +# successfully insert: max key has not reached maximum +run_sql 'INSERT INTO db.test(b) VALUES(11);' +# fail for insertion: db.test1 has key int64 - 1 +run_sql 'INSERT INTO db.test1(b) VALUES(22);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" check_contains 'ERROR' -# auto_random_max: 2^{64-1-10} - 1 = 2^53 - 1 -# sql succeed: though 9007199254740992 (2^53) is out-of-range of max_auto_random_base -# but lightning sets it to max_auto_random_base instead of 9007199254740991 -run_sql 'INSERT INTO db.test1(b) VALUES(22);' cleanup diff --git a/br/tests/lightning_max_random/config.toml b/br/tests/lightning_max_random/config.toml new file mode 100644 index 0000000000000..d2152b47c922a --- /dev/null +++ b/br/tests/lightning_max_random/config.toml @@ -0,0 +1,2 @@ +[tikv-importer] +backend = 'local' diff --git a/br/tests/lightning_max_random/data/db-schema-create.sql b/br/tests/lightning_max_random/data/db-schema-create.sql new file mode 100644 index 0000000000000..c88b0e3150e76 --- /dev/null +++ b/br/tests/lightning_max_random/data/db-schema-create.sql @@ -0,0 +1 @@ +create database db; \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test-schema.sql b/br/tests/lightning_max_random/data/db.test-schema.sql new file mode 100644 index 0000000000000..eb1838507ea94 --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test-schema.sql @@ -0,0 +1,5 @@ +create table test( + a bigint auto_random(10), + b int, + primary key(a) +); \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test.000000000.csv b/br/tests/lightning_max_random/data/db.test.000000000.csv new file mode 100644 index 0000000000000..a278110306af9 --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test.000000000.csv @@ -0,0 +1,3 @@ +"a","b" +1,2 +9007199254740990,3 \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test1-schema.sql b/br/tests/lightning_max_random/data/db.test1-schema.sql new file mode 100644 index 0000000000000..5ab8c5e9b32ef --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test1-schema.sql @@ -0,0 +1,5 @@ +create table test1( + a bigint auto_random(10), + b int, + primary key(a) +); \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test1.000000000.csv b/br/tests/lightning_max_random/data/db.test1.000000000.csv new file mode 100644 index 0000000000000..550f578de3e1a --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test1.000000000.csv @@ -0,0 +1,3 @@ +"a","b" +1,2 +9007199254740991,3 \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test2-schema.sql b/br/tests/lightning_max_random/data/db.test2-schema.sql new file mode 100644 index 0000000000000..740458dfd43df --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test2-schema.sql @@ -0,0 +1,5 @@ +create table test2( + a bigint auto_random(10), + b int, + primary key(a) +); \ No newline at end of file diff --git a/br/tests/lightning_max_random/data/db.test2.000000000.csv b/br/tests/lightning_max_random/data/db.test2.000000000.csv new file mode 100644 index 0000000000000..8de1e50edd09a --- /dev/null +++ b/br/tests/lightning_max_random/data/db.test2.000000000.csv @@ -0,0 +1,3 @@ +"a","b" +1,2 +9007199254740992,3 \ No newline at end of file diff --git a/br/tests/lightning_max_random/run.sh b/br/tests/lightning_max_random/run.sh new file mode 100644 index 0000000000000..972481e5bc0a5 --- /dev/null +++ b/br/tests/lightning_max_random/run.sh @@ -0,0 +1,65 @@ +#!/bin/sh +# +# Copyright 2022 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# 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 4 0 0 'local backend' || exit 0 + +ENGINE_COUNT=6 + +check_result() { + run_sql 'SHOW DATABASES;' + check_contains 'Database: db'; + run_sql 'SHOW TABLES IN db;' + check_contains 'Tables_in_db: test' + check_contains 'Tables_in_db: test1' + check_contains 'Tables_in_db: test2' + run_sql 'SELECT count(*) FROM db.test;' + check_contains 'count(*): 2' + run_sql 'SELECT count(*) FROM db.test1;' + check_contains 'count(*): 2' + run_sql 'SELECT count(*) FROM db.test2;' + check_contains 'count(*): 2' +} + +cleanup() { + rm -f $TEST_DIR/lightning.log + rm -rf $TEST_DIR/sst + run_sql 'DROP DATABASE IF EXISTS db;' +} + +cleanup + +# auto_random_max = 2^{64-1-10}-1 +# db.test contains key auto_random_max - 1 +# db.test1 contains key auto_random_max +# db.test2 contains key auto_random_max + 1 (overflow) +run_lightning --sorted-kv-dir "$TEST_DIR/sst" --config "tests/$TEST_NAME/config.toml" --log-file "$TEST_DIR/lightning.log" +check_result +# successfully insert: d.test auto_random key has not reached maximum +run_sql 'INSERT INTO db.test(b) VALUES(11);' +# fail for further insertion +run_sql 'INSERT INTO db.test(b) VALUES(22);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" +check_contains 'ERROR' +# fail: db.test1 has key auto_random_max +run_sql 'INSERT INTO db.test1(b) VALUES(11);' +run_sql 'INSERT INTO db.test1(b) VALUES(22);' 2>&1 | tee -a "$TEST_DIR/sql_res.$TEST_NAME.txt" +check_contains 'ERROR' +# successfully insert for overflow key +run_sql 'INSERT INTO db.test2(b) VALUES(33);' +run_sql 'INSERT INTO db.test2(b) VALUES(44);' +run_sql 'INSERT INTO db.test2(b) VALUES(55);' +cleanup From 6959514cbec6b5222a6e09bd5d166561709bbe9b Mon Sep 17 00:00:00 2001 From: buchuitoudegou <756541536@qq.com> Date: Thu, 28 Apr 2022 18:04:12 +0800 Subject: [PATCH 11/11] fix ut --- br/pkg/lightning/restore/tidb_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/br/pkg/lightning/restore/tidb_test.go b/br/pkg/lightning/restore/tidb_test.go index 5e261fd88d0f5..5d05b041e6fdb 100644 --- a/br/pkg/lightning/restore/tidb_test.go +++ b/br/pkg/lightning/restore/tidb_test.go @@ -435,6 +435,10 @@ func TestAlterAutoRandom(t *testing.T) { err := AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 12345, 288230376151711743) require.NoError(t, err) + // insert 288230376151711743 and try rebase to 288230376151711744 + err = AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", 288230376151711744, 288230376151711743) + require.NoError(t, err) + err = AlterAutoRandom(ctx, s.tiGlue.GetSQLExecutor(), "`db`.`table`", uint64(math.MaxInt64)+1, 288230376151711743) require.NoError(t, err) }