diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go
index 2040edaf7ab25..636afdaf24700 100644
--- a/br/pkg/backup/client.go
+++ b/br/pkg/backup/client.go
@@ -1233,9 +1233,9 @@ func (bc *Client) OnBackupResponse(
return txnlock.NewLock(lockErr), nil
}
}
- res := errContext.HandleIgnorableError(errPb, storeID)
+ res := utils.HandleBackupError(errPb, storeID, errContext)
switch res.Strategy {
- case utils.GiveUpStrategy:
+ case utils.StrategyGiveUp:
errMsg := res.Reason
if len(errMsg) <= 0 {
errMsg = errPb.Msg
diff --git a/br/pkg/utils/BUILD.bazel b/br/pkg/utils/BUILD.bazel
index 1b0272f37d956..0f4ceade484e6 100644
--- a/br/pkg/utils/BUILD.bazel
+++ b/br/pkg/utils/BUILD.bazel
@@ -7,10 +7,10 @@ go_library(
"db.go",
"dyn_pprof_other.go",
"dyn_pprof_unix.go",
+ "error_handling.go",
"json.go",
"key.go",
"misc.go",
- "permission.go",
"pointer.go",
"pprof.go",
"progress.go",
@@ -68,6 +68,7 @@ go_test(
srcs = [
"backoff_test.go",
"db_test.go",
+ "error_handling_test.go",
"json_test.go",
"key_test.go",
"main_test.go",
diff --git a/br/pkg/utils/backoff.go b/br/pkg/utils/backoff.go
index b9ce6dcb0b5d6..385ed4319a06a 100644
--- a/br/pkg/utils/backoff.go
+++ b/br/pkg/utils/backoff.go
@@ -172,8 +172,8 @@ func (bo *importerBackoffer) NextBackoff(err error) time.Duration {
// we don't care storeID here.
errs := multierr.Errors(err)
lastErr := errs[len(errs)-1]
- res := bo.errContext.HandleErrorMsg(lastErr.Error(), 0)
- if res.Strategy == RetryStrategy {
+ res := HandleUnknownBackupError(lastErr.Error(), 0, bo.errContext)
+ if res.Strategy == StrategyRetry {
bo.delayTime = 2 * bo.delayTime
bo.attempt--
} else {
diff --git a/br/pkg/utils/error_handling.go b/br/pkg/utils/error_handling.go
new file mode 100644
index 0000000000000..6c6c84a2a1884
--- /dev/null
+++ b/br/pkg/utils/error_handling.go
@@ -0,0 +1,196 @@
+// 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.
+
+package utils
+
+import (
+ "fmt"
+ "strings"
+ "sync"
+
+ backuppb "github.com/pingcap/kvproto/pkg/brpb"
+ "github.com/pingcap/log"
+ "go.uber.org/zap"
+)
+
+// UNSAFE! TODO: remove and map them to error types
+var retryableErrorMsg = []string{
+ "server closed",
+ "connection refused",
+ "connection reset by peer",
+ "channel closed",
+ "error trying to connect",
+ "connection closed before message completed",
+ "body write aborted",
+ "error during dispatch",
+ "put object timeout",
+ "timeout after",
+ "internalerror",
+ "not read from or written to within the timeout period",
+ "requesttimeout
",
+ "invalidpart
",
+ "end of file before message length reached",
+}
+
+// non-retryable error messages
+// UNSAFE! TODO: remove and map them to error types
+const (
+ ioMsg = "io"
+ notFoundMsg = "notfound"
+ permissionDeniedMsg = "permissiondenied"
+)
+
+// error messages
+const (
+ unreachableRetryMsg = "unreachable retry"
+ retryOnKvErrorMsg = "retry on kv error"
+ retryOnRegionErrorMsg = "retry on region error"
+ clusterIdMismatchMsg = "cluster id mismatch"
+ unknownErrorMsg = "unknown error"
+ contextCancelledMsg = "context canceled"
+ retryOnUnknownErrorMsg = "unknown error, retry it for a few times"
+ noRetryOnUnknownErrorMsg = "unknown error, retried too many times, give up"
+ retryableStorageErrorMsg = "retryable storage error"
+)
+
+type ErrorHandlingResult struct {
+ Strategy ErrorHandlingStrategy
+ Reason string
+}
+
+type ErrorHandlingStrategy int
+
+const (
+ // StrategyRetry error can be retried but will consume the backoff attempt quota.
+ StrategyRetry ErrorHandlingStrategy = iota
+ // StrategyGiveUp means unrecoverable error happened and the BR should exit
+ // for example:
+ // 1. permission not valid.
+ // 2. data not found.
+ // 3. retry too many times
+ StrategyGiveUp
+ // StrategyUnknown for StrategyUnknown error
+ StrategyUnknown
+)
+
+type ErrorContext struct {
+ mu sync.Mutex
+ // encounter times for one context on a store
+ // we may use this value to determine the retry policy
+ encounterTimes map[uint64]int
+ // unknown error retry limitation.
+ // encounter many times error makes Retry to GiveUp.
+ encounterTimesLimitation int
+ description string
+}
+
+func NewErrorContext(scenario string, limitation int) *ErrorContext {
+ return &ErrorContext{
+ description: scenario,
+ encounterTimes: make(map[uint64]int),
+ encounterTimesLimitation: limitation,
+ }
+}
+
+func NewDefaultContext() *ErrorContext {
+ return &ErrorContext{
+ description: "default",
+ encounterTimes: make(map[uint64]int),
+ encounterTimesLimitation: 1,
+ }
+}
+
+func HandleBackupError(err *backuppb.Error, storeId uint64, ec *ErrorContext) ErrorHandlingResult {
+ if err == nil {
+ return ErrorHandlingResult{StrategyRetry, unreachableRetryMsg}
+ }
+ res := handleBackupProtoError(err)
+ // try the best effort handle unknown error based on their error message
+ if res.Strategy == StrategyUnknown && len(err.Msg) != 0 {
+ return HandleUnknownBackupError(err.Msg, storeId, ec)
+ }
+ return res
+}
+
+func handleBackupProtoError(e *backuppb.Error) ErrorHandlingResult {
+ switch e.Detail.(type) {
+ case *backuppb.Error_KvError:
+ return ErrorHandlingResult{StrategyRetry, retryOnKvErrorMsg}
+ case *backuppb.Error_RegionError:
+ return ErrorHandlingResult{StrategyRetry, retryOnRegionErrorMsg}
+ case *backuppb.Error_ClusterIdError:
+ return ErrorHandlingResult{StrategyGiveUp, clusterIdMismatchMsg}
+ }
+ return ErrorHandlingResult{StrategyUnknown, unknownErrorMsg}
+}
+
+// HandleUnknownBackupError UNSAFE! TODO: remove this method and map all the current unknown errors to error types
+func HandleUnknownBackupError(msg string, uuid uint64, ec *ErrorContext) ErrorHandlingResult {
+ // UNSAFE! TODO: use meaningful error code instead of unstructured message to find failed to write error.
+ logger := log.L().With(zap.String("description", ec.description))
+ if messageIsNotFoundStorageError(msg) {
+ reason := fmt.Sprintf("File or directory not found on TiKV Node (store id: %v). "+
+ "workaround: please ensure br and tikv nodes share a same storage and the user of br and tikv has same uid.",
+ uuid)
+ return ErrorHandlingResult{StrategyGiveUp, reason}
+ }
+ if messageIsPermissionDeniedStorageError(msg) {
+ reason := fmt.Sprintf("I/O permission denied error occurs on TiKV Node(store id: %v). "+
+ "workaround: please ensure tikv has permission to read from & write to the storage.",
+ uuid)
+ return ErrorHandlingResult{StrategyGiveUp, reason}
+ }
+ msgLower := strings.ToLower(msg)
+ if strings.Contains(msgLower, contextCancelledMsg) {
+ return ErrorHandlingResult{StrategyGiveUp, contextCancelledMsg}
+ }
+
+ if MessageIsRetryableStorageError(msg) {
+ logger.Warn(retryableStorageErrorMsg, zap.String("error", msg))
+ return ErrorHandlingResult{StrategyRetry, retryableStorageErrorMsg}
+ }
+
+ // retry enough on same store
+ ec.mu.Lock()
+ defer ec.mu.Unlock()
+ ec.encounterTimes[uuid]++
+ if ec.encounterTimes[uuid] <= ec.encounterTimesLimitation {
+ return ErrorHandlingResult{StrategyRetry, retryOnUnknownErrorMsg}
+ }
+ return ErrorHandlingResult{StrategyGiveUp, noRetryOnUnknownErrorMsg}
+}
+
+// messageIsNotFoundStorageError checks whether the message returning from TiKV is "NotFound" storage I/O error
+func messageIsNotFoundStorageError(msg string) bool {
+ msgLower := strings.ToLower(msg)
+ return strings.Contains(msgLower, ioMsg) && strings.Contains(msgLower, notFoundMsg)
+}
+
+// MessageIsPermissionDeniedStorageError checks whether the message returning from TiKV is "PermissionDenied" storage I/O error
+func messageIsPermissionDeniedStorageError(msg string) bool {
+ msgLower := strings.ToLower(msg)
+ return strings.Contains(msgLower, permissionDeniedMsg)
+}
+
+// MessageIsRetryableStorageError checks whether the message returning from TiKV is retryable ExternalStorageError.
+func MessageIsRetryableStorageError(msg string) bool {
+ msgLower := strings.ToLower(msg)
+ // UNSAFE! TODO: Add a error type for retryable connection error.
+ for _, errStr := range retryableErrorMsg {
+ if strings.Contains(msgLower, errStr) {
+ return true
+ }
+ }
+ return false
+}
diff --git a/br/pkg/utils/error_handling_test.go b/br/pkg/utils/error_handling_test.go
new file mode 100644
index 0000000000000..6b82983843560
--- /dev/null
+++ b/br/pkg/utils/error_handling_test.go
@@ -0,0 +1,70 @@
+package utils
+
+import (
+ "testing"
+
+ backuppb "github.com/pingcap/kvproto/pkg/brpb"
+ "github.com/pingcap/kvproto/pkg/errorpb"
+ "github.com/stretchr/testify/require"
+)
+
+func TestHandleError(t *testing.T) {
+ ec := NewErrorContext("test", 3)
+ // Test case 1: Error is nil
+ result := HandleBackupError(nil, 123, ec)
+ require.Equal(t, ErrorHandlingResult{Strategy: StrategyRetry, Reason: unreachableRetryMsg}, result)
+
+ // Test case 2: Error is KvError and can be ignored
+ kvError := &backuppb.Error_KvError{}
+ result = HandleBackupError(&backuppb.Error{Detail: kvError}, 123, ec)
+ require.Equal(t, ErrorHandlingResult{Strategy: StrategyRetry, Reason: retryOnKvErrorMsg}, result)
+
+ // Test case 3: Error is RegionError and can be ignored
+ regionError := &backuppb.Error_RegionError{
+ RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{RegionId: 1}}}
+ result = HandleBackupError(&backuppb.Error{Detail: regionError}, 123, ec)
+ require.Equal(t, ErrorHandlingResult{Strategy: StrategyRetry, Reason: retryOnRegionErrorMsg}, result)
+
+ // Test case 4: Error is ClusterIdError
+ clusterIdError := &backuppb.Error_ClusterIdError{}
+ result = HandleBackupError(&backuppb.Error{Detail: clusterIdError}, 123, ec)
+ require.Equal(t, ErrorHandlingResult{Strategy: StrategyGiveUp, Reason: clusterIdMismatchMsg}, result)
+}
+
+func TestHandleErrorMsg(t *testing.T) {
+ ec := NewErrorContext("test", 3)
+
+ // Test messageIsNotFoundStorageError
+ msg := "IO: files Notfound error"
+ uuid := uint64(456)
+ expectedReason := "File or directory not found on TiKV Node (store id: 456). workaround: please ensure br and tikv nodes share a same storage and the user of br and tikv has same uid."
+ expectedResult := ErrorHandlingResult{Strategy: StrategyGiveUp, Reason: expectedReason}
+ actualResult := HandleUnknownBackupError(msg, uuid, ec)
+ require.Equal(t, expectedResult, actualResult)
+
+ // Test messageIsPermissionDeniedStorageError
+ msg = "I/O permissiondenied error occurs on TiKV Node(store id: 456)."
+ expectedReason = "I/O permission denied error occurs on TiKV Node(store id: 456). workaround: please ensure tikv has permission to read from & write to the storage."
+ expectedResult = ErrorHandlingResult{Strategy: StrategyGiveUp, Reason: expectedReason}
+ actualResult = HandleUnknownBackupError(msg, uuid, ec)
+ require.Equal(t, expectedResult, actualResult)
+
+ // Test MessageIsRetryableStorageError
+ msg = "server closed"
+ expectedResult = ErrorHandlingResult{Strategy: StrategyRetry, Reason: retryableStorageErrorMsg}
+ actualResult = HandleUnknownBackupError(msg, uuid, ec)
+ require.Equal(t, expectedResult, actualResult)
+
+ // Test unknown error
+ msg = "unknown error"
+ expectedResult = ErrorHandlingResult{Strategy: StrategyRetry, Reason: retryOnUnknownErrorMsg}
+ actualResult = HandleUnknownBackupError(msg, uuid, ec)
+ require.Equal(t, expectedResult, actualResult)
+
+ // Test retry too many times
+ _ = HandleUnknownBackupError(msg, uuid, ec)
+ _ = HandleUnknownBackupError(msg, uuid, ec)
+ expectedResult = ErrorHandlingResult{Strategy: StrategyGiveUp, Reason: noRetryOnUnknownErrorMsg}
+ actualResult = HandleUnknownBackupError(msg, uuid, ec)
+ require.Equal(t, expectedResult, actualResult)
+}
diff --git a/br/pkg/utils/permission.go b/br/pkg/utils/permission.go
deleted file mode 100644
index 3c0795db11c47..0000000000000
--- a/br/pkg/utils/permission.go
+++ /dev/null
@@ -1,20 +0,0 @@
-package utils
-
-import "strings"
-
-var (
- ioNotFoundMsg = "notfound"
- permissionDeniedMsg = "permissiondenied"
-)
-
-// messageIsNotFoundStorageError checks whether the message returning from TiKV is "NotFound" storage I/O error
-func messageIsNotFoundStorageError(msg string) bool {
- msgLower := strings.ToLower(msg)
- return strings.Contains(msgLower, "io") && strings.Contains(msgLower, ioNotFoundMsg)
-}
-
-// MessageIsPermissionDeniedStorageError checks whether the message returning from TiKV is "PermissionDenied" storage I/O error
-func messageIsPermissionDeniedStorageError(msg string) bool {
- msgLower := strings.ToLower(msg)
- return strings.Contains(msgLower, permissionDeniedMsg)
-}
diff --git a/br/pkg/utils/retry.go b/br/pkg/utils/retry.go
index 2efe37a8e8dff..6671f16b7842e 100644
--- a/br/pkg/utils/retry.go
+++ b/br/pkg/utils/retry.go
@@ -5,14 +5,11 @@ package utils
import (
"context"
stderrs "errors"
- "fmt"
- "strings"
"sync"
"time"
"github.com/google/uuid"
"github.com/pingcap/errors"
- backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/log"
tmysql "github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/parser/terror"
@@ -22,175 +19,6 @@ import (
"go.uber.org/zap"
)
-var retryableServerError = []string{
- "server closed",
- "connection refused",
- "connection reset by peer",
- "channel closed",
- "error trying to connect",
- "connection closed before message completed",
- "body write aborted",
- "error during dispatch",
- "put object timeout",
- "timeout after",
- "internalerror",
- "not read from or written to within the timeout period",
- "requesttimeout
",
- "invalidpart
",
- "end of file before message length reached",
-}
-
-type ErrorResult struct {
- Strategy ErrorStrategy
- Reason string
-}
-
-type ErrorStrategy int
-
-const (
- // This type can be retry but consume the backoffer attempts.
- RetryStrategy ErrorStrategy = iota
- // This type means unrecoverable error and the whole progress should exits
- // for example:
- // 1. permission not valid.
- // 2. data has not found.
- // 3. retry too many times
- GiveUpStrategy
- // This type represents Unknown error
- UnknownStrategy
-)
-
-type ErrorContext struct {
- mu sync.Mutex
- // encounter times for one context on a store
- // we may use this value to determine the retry policy
- encounterTimes map[uint64]int
- // unknown error retry limitation.
- // encouter many times error makes Retry to GiveUp.
- encounterTimesLimitation int
- // whether in backup or restore
- scenario string
-}
-
-func NewErrorContext(scenario string, limitation int) *ErrorContext {
- return &ErrorContext{
- scenario: scenario,
- encounterTimes: make(map[uint64]int),
- encounterTimesLimitation: limitation,
- }
-}
-
-func NewDefaultContext() *ErrorContext {
- return &ErrorContext{
- scenario: "default",
- encounterTimes: make(map[uint64]int),
- encounterTimesLimitation: 1,
- }
-}
-
-func (ec *ErrorContext) HandleError(err *backuppb.Error, uuid uint64) ErrorResult {
- if err == nil {
- return ErrorResult{RetryStrategy, "unreachable retry"}
- }
- res := ec.handleErrorPb(err, uuid)
- // try the best effort to save progress from error here
- if res.Strategy == UnknownStrategy && len(err.Msg) != 0 {
- return ec.HandleErrorMsg(err.Msg, uuid)
- }
- return res
-}
-
-func (ec *ErrorContext) HandleIgnorableError(err *backuppb.Error, uuid uint64) ErrorResult {
- if err == nil {
- return ErrorResult{RetryStrategy, "unreachable retry"}
- }
- res := ec.handleIgnorableErrorPb(err, uuid)
- // try the best effort to save progress from error here
- if res.Strategy == UnknownStrategy && len(err.Msg) != 0 {
- return ec.HandleErrorMsg(err.Msg, uuid)
- }
- return res
-}
-
-func (ec *ErrorContext) HandleErrorMsg(msg string, uuid uint64) ErrorResult {
- // UNSAFE! TODO: use meaningful error code instead of unstructured message to find failed to write error.
- logger := log.L().With(zap.String("scenario", ec.scenario))
- if messageIsNotFoundStorageError(msg) {
- reason := fmt.Sprintf("File or directory not found on TiKV Node (store id: %v). "+
- "work around:please ensure br and tikv nodes share a same storage and the user of br and tikv has same uid.",
- uuid)
- return ErrorResult{GiveUpStrategy, reason}
- }
- if messageIsPermissionDeniedStorageError(msg) {
- reason := fmt.Sprintf("I/O permission denied error occurs on TiKV Node(store id: %v). "+
- "work around:please ensure tikv has permission to read from & write to the storage.",
- uuid)
- return ErrorResult{GiveUpStrategy, reason}
- }
- msgLower := strings.ToLower(msg)
- if strings.Contains(msgLower, "context canceled") {
- return ErrorResult{GiveUpStrategy, "context canceled, give up"}
- }
-
- if MessageIsRetryableStorageError(msg) {
- logger.Warn("occur storage error", zap.String("error", msg))
- return ErrorResult{RetryStrategy, "retrable error"}
- }
- // retry enough on same store
- ec.mu.Lock()
- defer ec.mu.Unlock()
- ec.encounterTimes[uuid]++
- if ec.encounterTimes[uuid] <= ec.encounterTimesLimitation {
- return ErrorResult{RetryStrategy, "unknown error, retry it for few times"}
- }
- return ErrorResult{GiveUpStrategy, "unknown error and retry too many times, give up"}
-}
-
-func (ec *ErrorContext) handleIgnorableErrorPb(e *backuppb.Error, uuid uint64) ErrorResult {
- switch e.Detail.(type) {
- case *backuppb.Error_KvError:
- return ErrorResult{RetryStrategy, "retry outside because the error can be ignored"}
- case *backuppb.Error_RegionError:
- return ErrorResult{RetryStrategy, "retry outside because the error can be ignored"}
- case *backuppb.Error_ClusterIdError:
- return ErrorResult{GiveUpStrategy, "cluster ID mismatch"}
- }
- return ErrorResult{UnknownStrategy, "unreachable code"}
-}
-
-func (ec *ErrorContext) handleErrorPb(e *backuppb.Error, uuid uint64) ErrorResult {
- logger := log.L().With(zap.String("scenario", ec.scenario))
- switch v := e.Detail.(type) {
- case *backuppb.Error_KvError:
- // should not meet error other than KeyLocked.
- return ErrorResult{GiveUpStrategy, "unknown kv error"}
-
- case *backuppb.Error_RegionError:
- regionErr := v.RegionError
- // Ignore following errors.
- if !(regionErr.EpochNotMatch != nil ||
- regionErr.NotLeader != nil ||
- regionErr.RegionNotFound != nil ||
- regionErr.ServerIsBusy != nil ||
- regionErr.StaleCommand != nil ||
- regionErr.StoreNotMatch != nil ||
- regionErr.ReadIndexNotReady != nil ||
- regionErr.ProposalInMergingMode != nil) {
- logger.Error("unexpect region error", zap.Reflect("RegionError", regionErr))
- return ErrorResult{GiveUpStrategy, "unknown kv error"}
- }
- logger.Warn("occur region error",
- zap.Reflect("RegionError", regionErr),
- zap.Uint64("uuid", uuid))
- return ErrorResult{RetryStrategy, "retrable error"}
-
- case *backuppb.Error_ClusterIdError:
- logger.Error("occur cluster ID error", zap.Reflect("error", v), zap.Uint64("uuid", uuid))
- return ErrorResult{GiveUpStrategy, "cluster ID mismatch"}
- }
- return ErrorResult{UnknownStrategy, "unreachable code"}
-}
-
// RetryableFunc presents a retryable operation.
type RetryableFunc func() error
@@ -281,18 +109,6 @@ func WithRetryReturnLastErr(
return lastErr
}
-// MessageIsRetryableStorageError checks whether the message returning from TiKV is retryable ExternalStorageError.
-func MessageIsRetryableStorageError(msg string) bool {
- msgLower := strings.ToLower(msg)
- // UNSAFE! TODO: Add a error type for retryable connection error.
- for _, errStr := range retryableServerError {
- if strings.Contains(msgLower, errStr) {
- return true
- }
- }
- return false
-}
-
func FallBack2CreateTable(err error) bool {
switch nerr := errors.Cause(err).(type) {
case *terror.Error:
diff --git a/br/pkg/utils/retry_test.go b/br/pkg/utils/retry_test.go
index 605b59fadd8b8..c2afe35f47741 100644
--- a/br/pkg/utils/retry_test.go
+++ b/br/pkg/utils/retry_test.go
@@ -9,8 +9,6 @@ import (
"time"
"github.com/pingcap/errors"
- backuppb "github.com/pingcap/kvproto/pkg/brpb"
- "github.com/pingcap/kvproto/pkg/errorpb"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/stretchr/testify/require"
@@ -73,74 +71,3 @@ func TestFailNowIf(t *testing.T) {
assert.Equal(time.Duration(0), bo.NextBackoff(annotatedErr))
assert.Equal(0, bo.Attempt())
}
-
-func TestHandleError(t *testing.T) {
- ec := utils.NewErrorContext("test", 3)
- // Test case 1: Error is nil
- result := ec.HandleError(nil, 123)
- require.Equal(t, utils.ErrorResult{utils.RetryStrategy, "unreachable retry"}, result)
-
- // Test case 2: Error is KvError and can be ignored
- kvError := &backuppb.Error_KvError{}
- result = ec.HandleIgnorableError(&backuppb.Error{Detail: kvError}, 123)
- require.Equal(t, utils.ErrorResult{utils.RetryStrategy, "retry outside because the error can be ignored"}, result)
-
- // Test case 3: Error is KvError and cannot be ignored
- result = ec.HandleError(&backuppb.Error{Detail: kvError}, 123)
- require.Equal(t, utils.ErrorResult{utils.GiveUpStrategy, "unknown kv error"}, result)
-
- // Test case 4: Error is RegionError and can be ignored
- regionError := &backuppb.Error_RegionError{
- RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{RegionId: 1}}}
- result = ec.HandleIgnorableError(&backuppb.Error{Detail: regionError}, 123)
- require.Equal(t, utils.ErrorResult{utils.RetryStrategy, "retry outside because the error can be ignored"}, result)
-
- // Test case 5: Error is RegionError and cannot be ignored
- regionError = &backuppb.Error_RegionError{
- RegionError: &errorpb.Error{DiskFull: &errorpb.DiskFull{}}}
- result = ec.HandleError(&backuppb.Error{Detail: regionError}, 123)
- require.Equal(t, utils.ErrorResult{utils.GiveUpStrategy, "unknown kv error"}, result)
-
- // Test case 6: Error is ClusterIdError
- clusterIdError := &backuppb.Error_ClusterIdError{}
- result = ec.HandleError(&backuppb.Error{Detail: clusterIdError}, 123)
- require.Equal(t, utils.ErrorResult{utils.GiveUpStrategy, "cluster ID mismatch"}, result)
-}
-
-func TestHandleErrorMsg(t *testing.T) {
- ec := utils.NewErrorContext("test", 3)
-
- // Test messageIsNotFoundStorageError
- msg := "IO: files Notfound error"
- uuid := uint64(456)
- expectedReason := "File or directory not found on TiKV Node (store id: 456). work around:please ensure br and tikv nodes share a same storage and the user of br and tikv has same uid."
- expectedResult := utils.ErrorResult{utils.GiveUpStrategy, expectedReason}
- actualResult := ec.HandleErrorMsg(msg, uuid)
- require.Equal(t, expectedResult, actualResult)
-
- // Test messageIsPermissionDeniedStorageError
- msg = "I/O permissiondenied error occurs on TiKV Node(store id: 456)."
- expectedReason = "I/O permission denied error occurs on TiKV Node(store id: 456). work around:please ensure tikv has permission to read from & write to the storage."
- expectedResult = utils.ErrorResult{utils.GiveUpStrategy, expectedReason}
- actualResult = ec.HandleErrorMsg(msg, uuid)
- require.Equal(t, expectedResult, actualResult)
-
- // Test MessageIsRetryableStorageError
- msg = "server closed"
- expectedResult = utils.ErrorResult{utils.RetryStrategy, "retrable error"}
- actualResult = ec.HandleErrorMsg(msg, uuid)
- require.Equal(t, expectedResult, actualResult)
-
- // Test unknown error
- msg = "unknown error"
- expectedResult = utils.ErrorResult{utils.RetryStrategy, "unknown error, retry it for few times"}
- actualResult = ec.HandleErrorMsg(msg, uuid)
- require.Equal(t, expectedResult, actualResult)
-
- // Test retry too many times
- _ = ec.HandleErrorMsg(msg, uuid)
- _ = ec.HandleErrorMsg(msg, uuid)
- expectedResult = utils.ErrorResult{utils.GiveUpStrategy, "unknown error and retry too many times, give up"}
- actualResult = ec.HandleErrorMsg(msg, uuid)
- require.Equal(t, expectedResult, actualResult)
-}