Skip to content

Commit 468c62b

Browse files
authored
br: add retry for raw kv client put (#58963) (#59081)
close #58845
1 parent da5eed3 commit 468c62b

File tree

7 files changed

+167
-3
lines changed

7 files changed

+167
-3
lines changed

br/pkg/restore/client.go

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3176,7 +3176,7 @@ func (rc *Client) restoreMetaKvEntries(
31763176
failpoint.Inject("failed-to-restore-metakv", func(_ failpoint.Value) {
31773177
failpoint.Return(0, 0, errors.Errorf("failpoint: failed to restore metakv"))
31783178
})
3179-
if err := rc.rawKVClient.Put(ctx, newEntry.Key, newEntry.Value, entry.ts); err != nil {
3179+
if err := PutRawKvWithRetry(ctx, rc.rawKVClient, newEntry.Key, newEntry.Value, entry.ts); err != nil {
31803180
return 0, 0, errors.Trace(err)
31813181
}
31823182
// for failpoint, we need to flush the cache in rawKVClient every time
@@ -3802,3 +3802,13 @@ func (b *waitTiFlashBackoffer) NextBackoff(error) time.Duration {
38023802
func (b *waitTiFlashBackoffer) Attempt() int {
38033803
return b.Attempts
38043804
}
3805+
3806+
func PutRawKvWithRetry(ctx context.Context, client *RawKVBatchClient, key, value []byte, originTs uint64) error {
3807+
err := utils.WithRetry(ctx, func() error {
3808+
return client.Put(ctx, key, value, originTs)
3809+
}, utils.NewRawClientBackoffStrategy())
3810+
if err != nil {
3811+
return errors.Errorf("failed to put raw kv after retry")
3812+
}
3813+
return nil
3814+
}

br/pkg/restore/client_test.go

Lines changed: 67 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import (
3333
"github.com/pingcap/tidb/pkg/tablecodec"
3434
filter "github.com/pingcap/tidb/pkg/util/table-filter"
3535
"github.com/stretchr/testify/require"
36+
"github.com/tikv/client-go/v2/rawkv"
3637
pd "github.com/tikv/pd/client"
3738
"google.golang.org/grpc/keepalive"
3839
)
@@ -1928,3 +1929,69 @@ func TestCheckNewCollationEnable(t *testing.T) {
19281929
require.Equal(t, ca.newCollationEnableInCluster == "True", enabled)
19291930
}
19301931
}
1932+
1933+
type mockRawKVClient struct {
1934+
rawkv.Client
1935+
putCount int
1936+
errThreshold int
1937+
}
1938+
1939+
func (m *mockRawKVClient) BatchPut(ctx context.Context, keys, values [][]byte, options ...rawkv.RawOption) error {
1940+
m.putCount += 1
1941+
if m.errThreshold >= m.putCount {
1942+
return errors.New("rpcClient is idle")
1943+
}
1944+
return nil
1945+
}
1946+
1947+
func TestPutRawKvWithRetry(t *testing.T) {
1948+
tests := []struct {
1949+
name string
1950+
errThreshold int
1951+
cancelAfter time.Duration
1952+
wantErr string
1953+
wantPuts int
1954+
}{
1955+
{
1956+
name: "success on first try",
1957+
errThreshold: 0,
1958+
wantPuts: 1,
1959+
},
1960+
{
1961+
name: "success on after failure",
1962+
errThreshold: 2,
1963+
wantPuts: 3,
1964+
},
1965+
{
1966+
name: "fails all retries",
1967+
errThreshold: 5,
1968+
wantErr: "failed to put raw kv after retry",
1969+
wantPuts: 5,
1970+
},
1971+
}
1972+
1973+
for _, tt := range tests {
1974+
t.Run(tt.name, func(t *testing.T) {
1975+
mockRawClient := &mockRawKVClient{
1976+
errThreshold: tt.errThreshold,
1977+
}
1978+
client := restore.NewRawKVBatchClient(mockRawClient, 1)
1979+
1980+
ctx := context.Background()
1981+
if tt.cancelAfter > 0 {
1982+
var cancel context.CancelFunc
1983+
ctx, cancel = context.WithTimeout(ctx, tt.cancelAfter)
1984+
defer cancel()
1985+
}
1986+
1987+
err := restore.PutRawKvWithRetry(ctx, client, []byte("key"), []byte("value"), 1)
1988+
1989+
if tt.wantErr != "" {
1990+
require.ErrorContains(t, err, tt.wantErr)
1991+
} else {
1992+
require.NoError(t, err)
1993+
}
1994+
require.Equal(t, tt.wantPuts, mockRawClient.putCount)
1995+
})
1996+
}
1997+
}

br/pkg/restore/log_client/BUILD.bazel

Whitespace-only changes.

br/pkg/utils/backoff.go

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,10 @@ const (
5252
ChecksumMaxWaitInterval = 30 * time.Second
5353

5454
gRPC_Cancel = "the client connection is closing"
55+
56+
rawClientMaxAttempts = 5
57+
rawClientDelayTime = 500 * time.Millisecond
58+
rawClientMaxDelayTime = 5 * time.Second
5559
)
5660

5761
// At least, there are two possible cancel() call,
@@ -300,3 +304,35 @@ func (bo *pdReqBackoffer) NextBackoff(err error) time.Duration {
300304
func (bo *pdReqBackoffer) Attempt() int {
301305
return bo.attempt
302306
}
307+
308+
type RawClientBackoffStrategy struct {
309+
Attempts int
310+
BaseBackoff time.Duration
311+
MaxBackoff time.Duration
312+
}
313+
314+
func NewRawClientBackoffStrategy() Backoffer {
315+
return &RawClientBackoffStrategy{
316+
Attempts: rawClientMaxAttempts,
317+
BaseBackoff: rawClientDelayTime,
318+
MaxBackoff: rawClientMaxDelayTime,
319+
}
320+
}
321+
322+
// NextBackoff returns a duration to wait before retrying again
323+
func (b *RawClientBackoffStrategy) NextBackoff(error) time.Duration {
324+
bo := b.BaseBackoff
325+
b.Attempts--
326+
if b.Attempts == 0 {
327+
return 0
328+
}
329+
b.BaseBackoff *= 2
330+
if b.BaseBackoff > b.MaxBackoff {
331+
b.BaseBackoff = b.MaxBackoff
332+
}
333+
return bo
334+
}
335+
336+
func (b *RawClientBackoffStrategy) Attempt() int {
337+
return b.Attempts
338+
}

br/tests/br_file_corruption/run.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,7 @@ for filename in $(find $TEST_DIR/$DB -name "*.sst_bak"); do
6868
mv "$filename" "${filename%_bak}"
6969
done
7070

71-
export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/snap_client/full-restore-validate-checksum=return(true)"
71+
export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/full-restore-validate-checksum=return(true)"
7272
restore_fail=0
7373
run_br --pd $PD_ADDR restore full -s "local://$TEST_DIR/$DB" --checksum=true || restore_fail=1
7474
export GO_FAILPOINTS=""

br/tests/br_test_utils.sh

Lines changed: 51 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,51 @@
1+
#!/bin/sh
2+
#
3+
# Copyright 2024 PingCAP, Inc.
4+
#
5+
# Licensed under the Apache License, Version 2.0 (the "License");
6+
# you may not use this file except in compliance with the License.
7+
# You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
17+
set -eux
18+
19+
wait_log_checkpoint_advance() {
20+
local task_name=${1:-$TASK_NAME}
21+
echo "wait for log checkpoint to advance for task: $task_name"
22+
sleep 10
23+
local current_ts=$(python3 -c "import time; print(int(time.time() * 1000) << 18)")
24+
echo "current ts: $current_ts"
25+
i=0
26+
while true; do
27+
# extract the checkpoint ts of the log backup task. If there is some error, the checkpoint ts should be empty
28+
log_backup_status=$(unset BR_LOG_TO_TERM && run_br --pd $PD_ADDR log status --task-name $task_name --json 2>br.log)
29+
echo "log backup status: $log_backup_status"
30+
local checkpoint_ts=$(echo "$log_backup_status" | head -n 1 | jq 'if .[0].last_errors | length == 0 then .[0].checkpoint else empty end')
31+
echo "checkpoint ts: $checkpoint_ts"
32+
33+
# check whether the checkpoint ts is a number
34+
if [ $checkpoint_ts -gt 0 ] 2>/dev/null; then
35+
if [ $checkpoint_ts -gt $current_ts ]; then
36+
echo "the checkpoint has advanced"
37+
break
38+
fi
39+
echo "the checkpoint hasn't advanced"
40+
i=$((i+1))
41+
if [ "$i" -gt 50 ]; then
42+
echo 'the checkpoint lag is too large'
43+
exit 1
44+
fi
45+
sleep 10
46+
else
47+
echo "TEST: [$TEST_NAME] failed to wait checkpoint advance!"
48+
exit 1
49+
fi
50+
done
51+
}

br/tests/run_group.sh

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ mkdir $COV_DIR
2020
# Putting multiple light tests together and heavy tests in a separate group.
2121
declare -A groups
2222
groups=(
23-
["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"
23+
["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_check_dup_table"
2424
["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"
2525
["G02"]="br_full_cluster_restore br_full_ddl br_full_index br_gcs br_history br_pitr_failpoint br_pitr_long_running_schema_loading"
2626
["G03"]='br_incompatible_tidb_config br_incremental br_incremental_ddl br_incremental_index br_pitr'

0 commit comments

Comments
 (0)