br: add option to test encryption for all br int tests (#56434)
close pingcap/tidb#56433
This commit is contained in:
3
Makefile
3
Makefile
@ -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
18
br/tests/BUILD.bazel
Normal 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"],
|
||||
)
|
||||
@ -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"
|
||||
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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).
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
55
br/tests/run_br
Executable 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 "$@"
|
||||
|
||||
@ -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
233
br/tests/utils.go
Normal 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
|
||||
}
|
||||
Reference in New Issue
Block a user