Skip to content

Commit

Permalink
br: download sst again with no-crypt parameters when met decryption e…
Browse files Browse the repository at this point in the history
…rror for downloadSST (pingcap#32847) (pingcap#33197)

close pingcap#32423
  • Loading branch information
ti-srebot authored and joccau committed Jun 15, 2022
1 parent 459917c commit ece9fd1
Show file tree
Hide file tree
Showing 4 changed files with 109 additions and 2 deletions.
15 changes: 15 additions & 0 deletions br/pkg/restore/import.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ import (
"bytes"
"context"
"crypto/tls"
"strings"
"sync"
"sync/atomic"
"time"
Expand Down Expand Up @@ -330,6 +331,14 @@ func (importer *FileImporter) Import(
log.Warn("the connection to TiKV has been cut by a neko, meow :3")
e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3")
})
if isDecryptSstErr(e) {
log.Info("fail to decrypt when download sst, try again with no-crypt", logutil.File(f))
if importer.isRawKvMode {
downloadMeta, e = importer.downloadRawKVSST(ctx, info, f, nil)
} else {
downloadMeta, e = importer.downloadSST(ctx, info, f, rewriteRules, nil)
}
}
if e != nil {
remainFiles = remainFiles[i:]
return errors.Trace(e)
Expand Down Expand Up @@ -626,3 +635,9 @@ func (importer *FileImporter) ingestSSTs(
resp, err := importer.importClient.MultiIngest(ctx, leader.GetStoreId(), req)
return resp, errors.Trace(err)
}

func isDecryptSstErr(err error) bool {
return err != nil &&
strings.Contains(err.Error(), "Engine Engine") &&
strings.Contains(err.Error(), "Corruption: Bad table magic number")
}
5 changes: 3 additions & 2 deletions br/pkg/utils/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,10 @@ import (
)

const (
// importSSTRetryTimes specifies the retry time. Its longest time is about 90s-100s.
importSSTRetryTimes = 16
importSSTWaitInterval = 10 * time.Millisecond
importSSTMaxWaitInterval = 1 * time.Second
importSSTWaitInterval = 40 * time.Millisecond
importSSTMaxWaitInterval = 10 * time.Second

downloadSSTRetryTimes = 8
downloadSSTWaitInterval = 1 * time.Second
Expand Down
79 changes: 79 additions & 0 deletions br/tests/br_crypter2/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,79 @@
#!/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 -eu
DB="$TEST_NAME"
TABLE="usertable"
DB_COUNT=3

function create_db_with_table(){
for i in $(seq $DB_COUNT); do
run_sql "CREATE DATABASE $DB${i};"
go-ycsb load mysql -P tests/$TEST_NAME/workload -p mysql.host=$TIDB_IP -p mysql.port=$TIDB_PORT -p mysql.user=root -p mysql.db=$DB${i}
done
}

function drop_db(){
for i in $(seq $DB_COUNT); do
run_sql "DROP DATABASE $DB${i};"
done
}

function check_db_row(){
for i in $(seq $DB_COUNT); do
row_count_new[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}')
done

fail=false
for i in $(seq $DB_COUNT); do
if [ "${row_count_ori[i]}" != "${row_count_new[i]}" ];then
fail=true
echo "TEST: [$TEST_NAME] fail on database $DB${i}"
fi
echo "database $DB${i} [original] row count: ${row_count_ori[i]}, [after br] row count: ${row_count_new[i]}"
done

if $fail; then
echo "TEST: [$TEST_NAME] failed!"
exit 1
fi
}

# Create dbs with table
create_db_with_table

# Get the original row count from dbs
for i in $(seq $DB_COUNT); do
row_count_ori[${i}]=$(run_sql "SELECT COUNT(*) FROM $DB${i}.$TABLE;" | awk '/COUNT/{print $2}')
done

# Test backup/restore with crypt for br
CRYPTER_METHOD=aes128-ctr
CRYPTER_KEY="0123456789abcdef0123456789abcdef"

export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/backup/noop-backup=100*return(1)"
run_br --pd $PD_ADDR backup full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \
--use-backupmeta-v2=true --check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY

drop_db

run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB/${CRYPTER_METHOD}_file" \
--check-requirements=false --crypter.method $CRYPTER_METHOD --crypter.key $CRYPTER_KEY

check_db_row

# Drop dbs finally
drop_db
12 changes: 12 additions & 0 deletions br/tests/br_crypter2/workload
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
recordcount=1000
operationcount=0
workload=core

readallfields=true

readproportion=0
updateproportion=0
scanproportion=0
insertproportion=0

requestdistribution=uniform

0 comments on commit ece9fd1

Please sign in to comment.