br: add option to test encryption for all br int tests (#56434)

close pingcap/tidb#56433
This commit is contained in:
Wenqi Mou
2024-10-11 12:39:49 -04:00
committed by GitHub
parent 0f6bddd454
commit 99080c87ad
15 changed files with 394 additions and 21 deletions

View File

@ -436,7 +436,8 @@ build_for_br_integration_test:
$(GOBUILD) $(RACE_FLAG) -o bin/gc br/tests/br_z_gc_safepoint/*.go && \
$(GOBUILD) $(RACE_FLAG) -o bin/fake-oauth tools/fake-oauth/main.go && \
$(GOBUILD) $(RACE_FLAG) -o bin/rawkv br/tests/br_rawkv/*.go && \
$(GOBUILD) $(RACE_FLAG) -o bin/txnkv br/tests/br_txn/*.go \
$(GOBUILD) $(RACE_FLAG) -o bin/txnkv br/tests/br_txn/*.go && \
$(GOBUILD) $(RACE_FLAG) -o bin/utils br/tests/utils.go \
) || (make failpoint-disable && exit 1)
@make failpoint-disable

18
br/tests/BUILD.bazel Normal file
View File

@ -0,0 +1,18 @@
load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library")
go_library(
name = "tests_lib",
srcs = ["utils.go"],
importpath = "github.com/pingcap/tidb/br/tests",
visibility = ["//visibility:public"],
deps = [
"@com_github_klauspost_compress//zstd",
"@com_github_spf13_cobra//:cobra",
],
)
go_binary(
name = "tests",
embed = [":tests_lib"],
visibility = ["//visibility:public"],
)

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS for this script as not working with brv4.0.8
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eu
DB="$TEST_NAME"

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS and ENABLE_ENCRYPTION_CHECK for this script
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eu
DB="$TEST_NAME"
TABLE="usertable"

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS and ENABLE_ENCRYPTION_CHECK for this script
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eu
DB="$TEST_NAME"
TABLE="usertable"

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS and ENABLE_ENCRYPTION_CHECK for this script
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eu
. run_services
CUR=$(cd "$(dirname "$0")" && pwd)
@ -418,7 +424,7 @@ test_backup_encrypted_restore_unencrypted
test_plaintext
test_plaintext_data_key
test_local_master_key
# some issue running in CI, will fix later
# localstack not working with older glibc version in our centos7 base image...
#test_aws_kms
#test_aws_kms_with_iam
test_mixed_full_encrypted_log_plain

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable encryption as not working with brv4.0.8
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eux
DB="$TEST_NAME"
TABLE="usertable"

View File

@ -71,19 +71,20 @@ fi
# backup full with ratelimit = 1 to make sure this backup task won't finish quickly
echo "backup start to test lock file"
PPROF_PORT=6080
# not easy to trap USR1 signal in our custom run_br, so we use the original run_br to run the backup command
GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/utils/determined-pprof-port=return($PPROF_PORT)" \
run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/lock" \
$UTILS_DIR/run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/lock" \
--remove-schedulers \
--ratelimit 1 \
--ratelimit-unit 1 \
--concurrency 4 &> $TEST_DIR/br-other-stdout.log & # It will be killed after test finish.
--ratelimit-unit 1 &> $TEST_DIR/br-other-stdout.log & # It will be killed after test finish.
# record last backup pid
_pid=$!
# give the former backup some time to write down lock file (and initialize signal listener).
sleep 1
pkill -10 -P $_pid
# don't use numeric value as it's mapped differently on different unix like system
pkill -USR1 -P $_pid
echo "starting pprof..."
# give the former backup some time to write down lock file (and start pprof server).

View File

@ -72,7 +72,7 @@ echo "incremental_delete_range_count: $incremental_delete_range_count"
# wait checkpoint advance
echo "wait checkpoint advance"
sleep 10
current_ts=$(echo $(($(date +%s%3N) << 18)))
current_ts=$(python3 -c "import time; print(int(time.time() * 1000) << 18)")
echo "current ts: $current_ts"
i=0
while true; do
@ -122,7 +122,7 @@ check_result() {
}
# start a new cluster
echo "restart a services"
echo "restart services"
restart_services
# non-compliant operation
@ -130,7 +130,7 @@ echo "non compliant operation"
restore_fail=0
run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --start-ts $current_ts || restore_fail=1
if [ $restore_fail -ne 1 ]; then
echo 'pitr success'
echo 'pitr success on non compliant operation'
exit 1
fi
@ -142,7 +142,7 @@ run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --full-bac
check_result
# start a new cluster for incremental + log
echo "restart a services"
echo "restart services"
restart_services
echo "run snapshot restore#2"
@ -154,7 +154,7 @@ run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --full-bac
check_result
# start a new cluster for incremental + log
echo "restart a services"
echo "restart services"
restart_services
echo "run snapshot restore#3"
@ -164,7 +164,7 @@ echo "run incremental restore but failed"
restore_fail=0
run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$PREFIX/inc_fail" || restore_fail=1
if [ $restore_fail -ne 1 ]; then
echo 'pitr success'
echo 'pitr success on incremental restore'
exit 1
fi
@ -186,7 +186,7 @@ restore_fail=0
run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --full-backup-storage "local://$TEST_DIR/$PREFIX/full" || restore_fail=1
export GO_FAILPOINTS=""
if [ $restore_fail -ne 1 ]; then
echo 'pitr success'
echo 'pitr success on file lost'
exit 1
fi
@ -198,6 +198,6 @@ restore_fail=0
run_br --pd $PD_ADDR restore point -s "local://$TEST_DIR/$PREFIX/log" --full-backup-storage "local://$TEST_DIR/$PREFIX/full" || restore_fail=1
export GO_FAILPOINTS=""
if [ $restore_fail -ne 1 ]; then
echo 'pitr success'
echo 'pitr success on file corruption'
exit 1
fi

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS and ENABLE_ENCRYPTION_CHECK for this script
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eux
# restart service without tiflash

View File

@ -14,6 +14,12 @@
# See the License for the specific language governing permissions and
# limitations under the License.
# disable global ENCRYPTION_ARGS and ENABLE_ENCRYPTION_CHECK for this script
ENCRYPTION_ARGS=""
ENABLE_ENCRYPTION_CHECK=false
export ENCRYPTION_ARGS
export ENABLE_ENCRYPTION_CHECK
set -eux
# restart service without tiflash

View File

@ -17,7 +17,7 @@
set -eu
CUR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
export UTILS_DIR="$CUR/../../tests/_utils"
export PATH="$PATH:$CUR/../../bin:$CUR/../bin:$UTILS_DIR"
export PATH="$CUR:$PATH:$CUR/../../bin:$CUR/../bin:$UTILS_DIR"
export TEST_DIR=/tmp/backup_restore_test
export COV_DIR="/tmp/group_cover"
mkdir -p $COV_DIR || true
@ -48,6 +48,25 @@ start_services $@
run_curl "https://$PD_ADDR/pd/api/v1/version" | grep -o 'v[0-9.]\+' > "$TEST_DIR/cluster_version.txt"
IFS='.' read CLUSTER_VERSION_MAJOR CLUSTER_VERSION_MINOR CLUSTER_VERSION_REVISION < "$TEST_DIR/cluster_version.txt"
# enable encryption validation for all tests if needed, it will check all backup files are encrypted or not
# based on the encryption settings in the test case
ENABLE_ENCRYPTION_CHECK=true
export ENABLE_ENCRYPTION_CHECK
# generate local disk master key file to enable local encryption
MASTER_KEY_DIR="$TEST_DIR/master_key"
mkdir -p "$MASTER_KEY_DIR"
openssl rand -hex 32 > "$MASTER_KEY_DIR/master.key"
MASTER_KEY_PATH="local://$MASTER_KEY_DIR/master.key"
# testing encryption using local master key
ENCRYPTION_ARGS="--crypter.method aes128-ctr --crypter.key 0123456789abcdef0123456789abcdef --master-key-crypter-method AES256-CTR --master-key $MASTER_KEY_PATH"
# uncomment it to test plaintext data key manually.
#ENCRYPTION_ARGS="--crypter.method aes128-ctr --crypter.key 0123456789abcdef0123456789abcdef --log.crypter.method AES256-CTR --log.crypter.key 0123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"
export ENCRYPTION_ARGS
if [ "${1-}" = '--debug' ]; then
echo 'You may now debug from another terminal. Press [ENTER] to continue.'
read line

55
br/tests/run_br Executable file
View File

@ -0,0 +1,55 @@
#!/bin/sh
# Copyright 2024 PingCAP, Inc. Licensed under Apache-2.0.
set -eux
main() {
# Call the original run.sh script
"$UTILS_DIR/run_br" "$@" $ENCRYPTION_ARGS
# Check if encryption validation is enabled
if [ "${ENABLE_ENCRYPTION_CHECK:-false}" != "true" ]; then
echo "Encryption check is disabled. Skipping backup file encryption validation."
exit 0
fi
echo "Starting backup file encryption validation..."
# Capture the output of the validation command
output=$(bin/utils validateBackupFiles --command="$*" --encryption="$ENCRYPTION_ARGS")
exit_code=$?
if [ $exit_code -ne 0 ]; then
echo "Validation failed. Exiting with status 1."
exit 1
fi
# Check if validation is needed
# strings are from utils.go
if echo "$output" | grep -q "No need to validate"; then
echo "Validation not required. Skipping."
exit 0
fi
# Check for expected strings in the output
if [ -n "$ENCRYPTION_ARGS" ]; then
# strings are from utils.go
if ! echo "$output" | grep -q "All files in .* are encrypted, as expected with encryption"; then
echo "Error: Expected 'All files are encrypted' message not found in output with encryption"
exit 1
fi
else
# strings are from utils.go
if ! echo "$output" | grep -q "All files in .* are not encrypted, as expected without encryption"; then
echo "Error: Expected 'All files are not encrypted' message not found in output without encryption"
exit 1
fi
fi
echo "Validation completed successfully."
}
# Execute the main function
main "$@"

View File

@ -20,15 +20,15 @@ mkdir -p $COV_DIR
# Putting multiple light tests together and heavy tests in a separate group.
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"
["G02"]="br_full_cluster_restore br_full_index br_incremental_ddl br_pitr_failpoint br_pitr_gc_safepoint"
["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 br_tiflash_conflict'
["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 br_tidb_placement_policy"
["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 br_tiflash"
["G02"]="br_full_cluster_restore br_full_index br_incremental_ddl br_pitr_failpoint br_pitr_gc_safepoint br_other"
["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'
["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'
["G05"]='br_skip_checksum br_split_region_fail br_systables br_table_filter br_txn br_stats br_clustered_index br_crypter'
["G05"]='br_skip_checksum br_split_region_fail br_systables br_table_filter br_txn br_stats br_clustered_index br_crypter br_partition_add_index'
["G06"]='br_tikv_outage br_tikv_outage3 br_restore_checkpoint br_encryption'
["G07"]='br_pitr'
["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint br_autorandom br_file_corruption'
["G08"]='br_tikv_outage2 br_ttl br_views_and_sequences br_z_gc_safepoint br_autorandom br_file_corruption br_tiflash_conflict'
)
# Get other cases not in groups, to avoid missing any case
@ -44,6 +44,10 @@ for script in "$CUR"/*/run.sh; do
fi
done
# enable local encryption for all tests
ENABLE_ENCRYPTION=true
export ENABLE_ENCRYPTION
if [[ "$group" == "others" ]]; then
if [[ -z $others ]]; then
echo "All br integration test cases have been added to groups"

233
br/tests/utils.go Normal file
View File

@ -0,0 +1,233 @@
// Copyright 2024 PingCAP, Inc. Licensed under Apache-2.0.
package main
import (
"encoding/binary"
"fmt"
"io"
"os"
"path/filepath"
"strings"
"github.com/klauspost/compress/zstd"
"github.com/spf13/cobra"
)
const (
cmdValidateBackupFiles = "validateBackupFiles"
extSST = ".sst"
extLOG = ".log"
)
func main() {
rootCmd := &cobra.Command{
Use: "utils",
Short: "Utility commands for backup and restore",
}
validateCmd := &cobra.Command{
Use: cmdValidateBackupFiles,
Short: "Validate backup files",
Run: runValidateBackupFiles,
}
validateCmd.Flags().String("command", "", "Backup or restore command")
validateCmd.Flags().String("encryption", "", "Encryption argument")
rootCmd.AddCommand(validateCmd)
if err := rootCmd.Execute(); err != nil {
fmt.Println(err)
os.Exit(1)
}
}
func runValidateBackupFiles(cmd *cobra.Command, args []string) {
command, _ := cmd.Flags().GetString("command")
encryptionArg, _ := cmd.Flags().GetString("encryption")
if command == "" {
fmt.Println("Please provide the full backup or restore command using --command flag")
err := cmd.Usage()
if err != nil {
fmt.Println("Usage error")
return
}
os.Exit(1)
}
storagePath, found := parseCommand(command)
// doesn't need to validate if it's not doing backup/restore
if !found {
fmt.Println("No need to validate")
return
}
fmt.Printf("Validating files in: %s\n", storagePath)
if !checkCompressionAndEncryption(storagePath, encryptionArg) {
fmt.Println("validation failed")
os.Exit(1)
}
}
// parseCommand parses the command and only returns the storage path if it's a full backup or restore point
// as full backup will have backup files ready in the storage path after returning from the command
// and log backup will not, so we can only use restore point to validate.
func parseCommand(cmd string) (string, bool) {
// Create a temporary cobra command to parse the input
tempCmd := &cobra.Command{}
tempCmd.Flags().String("s", "", "Storage path (short)")
tempCmd.Flags().String("storage", "", "Storage path (long)")
// Split the command string into args
args := strings.Fields(cmd)
// Parse the args
if err := tempCmd.Flags().Parse(args); err != nil {
return "", false
}
// Check for backup or restore point command
hasBackupOrRestorePoint := false
for i, arg := range args {
if arg == "backup" {
hasBackupOrRestorePoint = true
break
}
if i < len(args)-1 && arg == "restore" && args[i+1] == "point" {
hasBackupOrRestorePoint = true
break
}
}
// Get the storage path from either -s or -storage flag
storagePath, _ := tempCmd.Flags().GetString("s")
if storagePath == "" {
storagePath, _ = tempCmd.Flags().GetString("storage")
}
storagePath = strings.TrimPrefix(storagePath, "local://")
if hasBackupOrRestorePoint && storagePath != "" {
return storagePath, true
}
return "", false
}
func checkCompressionAndEncryption(dir string, encryptionArg string) bool {
allEncrypted := true
allUnencrypted := true
totalFiles := 0
err := filepath.Walk(dir, func(path string, info os.FileInfo, err error) error {
if err != nil {
return err
}
if info.IsDir() {
return nil
}
if strings.HasSuffix(path, extSST) {
totalFiles++
isValidSST, err := isLikelySSTFile(path)
if err != nil {
fmt.Printf("Error checking SST file %s: %v\n", path, err)
return err
}
if isValidSST {
allEncrypted = false
} else {
allUnencrypted = false
}
} else if strings.HasSuffix(path, extLOG) {
totalFiles++
isCompressed, err := isZstdCompressed(path)
if err != nil {
fmt.Printf("Error checking if file is encrypted %s: %v\n", path, err)
return err
}
if isCompressed {
allEncrypted = false
} else {
allUnencrypted = false
}
}
return nil
})
if err != nil {
fmt.Printf("Error walking through directory: %v\n", err)
os.Exit(1)
}
// handle with encryption case
if encryptionArg != "" {
if allEncrypted {
fmt.Printf("All files in %s are encrypted, as expected with encryption\n", dir)
return true
}
fmt.Printf("Error: Some files in %s are not encrypted, which is unexpected with encryption\n", dir)
return false
}
// handle without encryption case
if allUnencrypted {
fmt.Printf("All files in %s are not encrypted, as expected without encryption\n", dir)
return true
} else if allEncrypted {
fmt.Printf("Error: All files in %s are encrypted, which is unexpected without encryption\n", dir)
return false
}
fmt.Printf("Error: Mixed encryption in %s. Some files are encrypted, some are not.\n", dir)
return false
}
func isZstdCompressed(filePath string) (bool, error) {
file, err := os.OpenFile(filePath, os.O_RDONLY, 0) //nolint:gosec
if err != nil {
return false, err
}
defer file.Close()
decoder, err := zstd.NewReader(file)
if err != nil {
return false, nil // Not compressed or error in compression
}
defer decoder.Close()
// Try to read a small amount of data
_, err = decoder.Read(make([]byte, 1))
if err != nil {
return false, nil // Not compressed or error in decompression
}
return true, nil
}
func isLikelySSTFile(filePath string) (bool, error) {
file, err := os.OpenFile(filePath, os.O_RDONLY, 0) //nolint:gosec
if err != nil {
return false, err
}
defer file.Close()
// Seek to 8 bytes from the end of the file
_, err = file.Seek(-8, io.SeekEnd)
if err != nil {
return false, err
}
// Read the last 8 bytes
footer := make([]byte, 8)
_, err = file.Read(footer)
if err != nil {
return false, err
}
// Check for SST magic number (kLegacyBlockBasedTableMagicNumber)
// or (kBlockBasedTableMagicNumber)
magicNumber := binary.LittleEndian.Uint64(footer)
return magicNumber == 0xdb4775248b80fb57 || magicNumber == 0x88e241b785f4cff7, nil
}