Skip to content

Commit

Permalink
br: make table existence check unified on different br client (pingca…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot authored Dec 13, 2024
1 parent db9a6b9 commit c44264f
Show file tree
Hide file tree
Showing 6 changed files with 101 additions and 8 deletions.
1 change: 1 addition & 0 deletions br/pkg/task/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ go_library(
"//br/pkg/utils",
"//br/pkg/version",
"//pkg/config",
"//pkg/infoschema",
"//pkg/kv",
"//pkg/parser/model",
"//pkg/parser/mysql",
Expand Down
24 changes: 24 additions & 0 deletions br/pkg/task/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/br/pkg/version"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/infoschema"
"github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/mathutil"
"github.com/spf13/cobra"
Expand Down Expand Up @@ -845,6 +846,11 @@ func runRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf
if cfg.WithSysTable {
client.InitFullClusterRestore(cfg.ExplicitFilter)
}
} else if client.IsFull() && checkpointFirstRun && cfg.CheckRequirements {
if err := checkTableExistence(ctx, mgr, tables, g); err != nil {
schedulersRemovable = true
return errors.Trace(err)
}
}

if client.IsFullClusterRestore() && client.HasBackedUpSysDB() {
Expand Down Expand Up @@ -1137,6 +1143,24 @@ func runRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf
return nil
}

func checkTableExistence(ctx context.Context, mgr *conn.Mgr, tables []*metautil.Table, g glue.Glue) error {
message := "table already exists: "
allUnique := true
for _, table := range tables {
_, err := mgr.GetDomain().InfoSchema().TableByName(table.DB.Name, table.Info.Name)
if err == nil {
message += fmt.Sprintf("%s.%s ", table.DB.Name, table.Info.Name)
allUnique = false
} else if !infoschema.ErrTableNotExists.Equal(err) {
return errors.Trace(err)
}
}
if !allUnique {
return errors.Errorf("Target tables already exist: %s", message)
}
return nil
}

// dropToBlackhole drop all incoming tables into black hole,
// i.e. don't execute checksum, just increase the process anyhow.
func dropToBlackhole(
Expand Down
10 changes: 5 additions & 5 deletions br/tests/br_300_small_tables/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -73,16 +73,15 @@ else
fi

# truncate every table
# (FIXME: drop instead of truncate. if we drop then create-table will still be executed and wastes time executing DDLs)
i=1
while [ $i -le $TABLES_COUNT ]; do
run_sql "truncate $DB.sbtest$i;"
run_sql "drop table $DB.sbtest$i;"
i=$(($i+1))
done

rm -rf $RESTORE_LOG
echo "restore 1/300 of the table start..."
run_br restore table --db $DB --table "sbtest100" --log-file $RESTORE_LOG -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --no-schema
run_br restore table --db $DB --table "sbtest100" --log-file $RESTORE_LOG -s "local://$TEST_DIR/$DB" --pd $PD_ADDR
restore_size=`grep "restore-data-size" "${RESTORE_LOG}" | grep -oP '\[\K[^\]]+' | grep "restore-data-size" | awk -F '=' '{print $2}' | grep -oP '\d*\.?\d+'`
echo "restore data size is ${restore_size}"

Expand All @@ -98,9 +97,10 @@ else
exit 1
fi

run_sql "drop table $DB.sbtest100;"

# restore db
# (FIXME: shouldn't need --no-schema to be fast, currently the alter-auto-id DDL slows things down)
echo "restore start..."
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --no-schema
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR

run_sql "DROP DATABASE $DB;"
68 changes: 68 additions & 0 deletions br/tests/br_check_dup_table/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
#!/bin/sh
#
# 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
DB="$TEST_NAME"

run_sql "CREATE DATABASE $DB;"

run_sql "CREATE TABLE $DB.usertable1 ( \
YCSB_KEY varchar(64) NOT NULL, \
FIELD0 varchar(1) DEFAULT NULL, \
PRIMARY KEY (YCSB_KEY) \
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;"

run_sql "INSERT INTO $DB.usertable1 VALUES (\"a\", \"b\");"
run_sql "INSERT INTO $DB.usertable1 VALUES (\"aa\", \"b\");"

run_sql "CREATE TABLE $DB.usertable2 ( \
YCSB_KEY varchar(64) NOT NULL, \
FIELD0 varchar(1) DEFAULT NULL, \
PRIMARY KEY (YCSB_KEY) \
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;"

run_sql "INSERT INTO $DB.usertable2 VALUES (\"c\", \"d\");"
# backup db
echo "backup start..."
run_br --pd $PD_ADDR backup db --db "$DB" -s "local://$TEST_DIR/$DB"

run_sql "DROP DATABASE $DB;"

# restore db
echo "restore start..."
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR

table_count=$(run_sql "use $DB; show tables;" | grep "Tables_in" | wc -l)
if [ "$table_count" -ne "2" ];then
echo "TEST: [$TEST_NAME] failed!"
exit 1
fi

# restore db again
echo "restore start..."
LOG_OUTPUT=$(run_br restore db --db "$DB" -s "local://$TEST_DIR/$DB" --pd "$PD_ADDR" 2>&1 || true)

# Check if the log contains 'ErrTableAlreadyExisted'
if ! echo "$LOG_OUTPUT" | grep -q "BR:Restore:ErrTablesAlreadyExisted"; then
echo "Error: 'ErrTableAlreadyExisted' not found in logs."
echo "Log output:"
echo "$LOG_OUTPUT"
exit 1
else
echo "restore failed as expect"
fi

run_sql "DROP DATABASE $DB;"
4 changes: 2 additions & 2 deletions br/tests/br_incompatible_tidb_config/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -54,14 +54,14 @@ run_br --pd $PD_ADDR backup db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_
run_sql "drop schema $DB;"
# restore with ddl(create table) job one by one
run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$TABLE" --ddl-batch-size=1

run_sql "drop schema $DB;"
run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_TABLE" --ddl-batch-size=1

# restore
run_sql "drop schema $DB;"
# restore with batch create table
run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$TABLE" --ddl-batch-size=128

run_sql "drop schema $DB;"
run_br --pd $PD_ADDR restore db --db "$DB" -s "local://$TEST_DIR/$DB$INCREMENTAL_TABLE" --ddl-batch-size=128

run_sql "drop schema $DB;"
Expand Down
2 changes: 1 addition & 1 deletion br/tests/run_group_br_tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ mkdir -p $COV_DIR
declare -A groups
groups=(
["G00"]="br_300_small_tables br_backup_empty br_backup_version br_cache_table br_case_sensitive br_charset_gbk br_check_new_collocation_enable br_history br_gcs br_rawkv"
["G01"]="br_autoid br_crypter2 br_db br_db_online br_db_online_newkv br_db_skip br_debug_meta br_ebs br_foreign_key br_full br_table_partition br_full_ddl"
["G01"]="br_autoid br_crypter2 br_db br_check_dup_table br_db_online br_db_online_newkv br_db_skip br_debug_meta br_ebs br_foreign_key br_full br_table_partition br_full_ddl"
["G02"]="br_full_cluster_restore br_full_index br_incremental_ddl br_pitr_failpoint br_pitr_long_running_schema_loading"
["G03"]='br_incompatible_tidb_config br_incremental br_incremental_index br_incremental_only_ddl br_incremental_same_table br_insert_after_restore br_key_locked br_log_test br_move_backup br_mv_index br_other br_partition_add_index br_tidb_placement_policy br_tiflash'
["G04"]='br_range br_replica_read br_restore_TDE_enable br_restore_log_task_enable br_s3 br_shuffle_leader br_shuffle_region br_single_table'
Expand Down

0 comments on commit c44264f

Please sign in to comment.