Skip to content

Commit

Permalink
br: rebase auto random id if the table is common handle (#52256) (#52295
Browse files Browse the repository at this point in the history
)

close #52255
  • Loading branch information
ti-chi-bot authored Apr 2, 2024
1 parent df644d3 commit fde2234
Show file tree
Hide file tree
Showing 4 changed files with 74 additions and 3 deletions.
2 changes: 1 addition & 1 deletion br/pkg/backup/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -674,7 +674,7 @@ func BuildBackupSchemas(
// Treat cached table as normal table.
tableInfo.TableCacheStatusType = model.TableCacheStatusDisable

if tableInfo.PKIsHandle && tableInfo.ContainsAutoRandomBits() {
if tableInfo.ContainsAutoRandomBits() {
// this table has auto_random id, we need backup and rebase in restoration
var globalAutoRandID int64
globalAutoRandID, err = autoIDAccess.RandomID().Get()
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/restore/db.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,7 @@ func (db *DB) CreateTablePostRestore(ctx context.Context, table *metautil.Table,
utils.EncloseName(table.DB.Name.O),
utils.EncloseName(table.Info.Name.O),
table.Info.AutoIncID)
} else if table.Info.PKIsHandle && table.Info.ContainsAutoRandomBits() {
} else if table.Info.ContainsAutoRandomBits() {
restoreMetaSQL = fmt.Sprintf(
"alter table %s.%s auto_random_base = %d",
utils.EncloseName(table.DB.Name.O),
Expand Down
71 changes: 71 additions & 0 deletions br/tests/br_autorandom/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
#!/bin/bash
#
# Copyright 2024 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 -eu
. run_services
CUR=$(cd `dirname $0`; pwd)

# const value
PREFIX="autorandom" # NOTICE: don't start with 'br' because `restart services` would remove file/directory br*.
res_file="$TEST_DIR/sql_res.$TEST_NAME.txt"

# start a new cluster
echo "restart a services"
restart_services

# prepare the data
echo "prepare the data"
run_sql "CREATE TABLE test.common (a BIGINT UNSIGNED AUTO_RANDOM(1), b VARCHAR(255), uid INT, c VARCHAR(255) DEFAULT 'c', PRIMARY KEY (a, b), UNIQUE INDEX (uid));"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 1, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 2, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 3, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 4, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 5, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 6, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 7, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 8, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 9, 'a');"
run_sql "INSERT INTO test.common (b, uid, c) values ('a', 10, 'a');"

run_sql "CREATE TABLE test.pk (a BIGINT UNSIGNED AUTO_RANDOM(1), uid INT, c VARCHAR(255) DEFAULT 'c', PRIMARY KEY (a), UNIQUE INDEX (uid));"
run_sql "INSERT INTO test.pk (uid, c) values (1, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (2, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (3, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (4, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (5, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (6, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (7, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (8, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (9, 'a');"
run_sql "INSERT INTO test.pk (uid, c) values (10, 'a');"

# backup & restore
run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$PREFIX/full"
echo "restart a services"
restart_services
run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$PREFIX/full"

# new workload
for i in `seq 1 9`; do
run_sql "INSERT INTO test.common (b, uid) values ('a', 10) on duplicate key update c = 'b';"
run_sql "INSERT INTO test.pk (uid) values (10) on duplicate key update c = 'b';"
done

# check consistency
run_sql "SELECT COUNT(*) AS RESCNT FROM test.common WHERE uid < 10 AND c = 'b';"
check_contains "RESCNT: 0"
run_sql "SELECT COUNT(*) AS RESCNT FROM test.pk WHERE uid < 10 AND c = 'b';"
check_contains "RESCNT: 0"
2 changes: 1 addition & 1 deletion br/tests/run_group.sh
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ groups=(
["G05"]='br_range br_rawkv br_replica_read br_restore_TDE_enable br_restore_log_task_enable br_s3 br_shuffle_leader br_shuffle_region br_single_table'
["G06"]='br_skip_checksum br_small_batch_size br_split_region_fail br_systables br_table_filter br_txn br_stats'
["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'
["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint br_autorandom 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'
Expand Down

0 comments on commit fde2234

Please sign in to comment.