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