diff --git a/errors.toml b/errors.toml index 6766da79572..e6025e5a759 100644 --- a/errors.toml +++ b/errors.toml @@ -381,6 +381,16 @@ error = ''' watcher canceled ''' +["PD:etcd:ErrExceedMaxEtcdTxnOps"] +error = ''' +exceed max etcd txn operations +''' + +["PD:etcd:ErrIllegalOperation"] +error = ''' +unknown operation +''' + ["PD:etcd:ErrNewEtcdClient"] error = ''' new etcd client failed @@ -471,6 +481,136 @@ error = ''' failed to unmarshal json ''' +["PD:keyspace:ErrKeyspaceExists"] +error = ''' +keyspace already exists +''' + +["PD:keyspace:ErrKeyspaceNotAssigned"] +error = ''' +the keyspace %d isn't assigned to any keyspace group +''' + +["PD:keyspace:ErrKeyspaceNotFound"] +error = ''' +keyspace does not exist +''' + +["PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup"] +error = ''' +keyspace is not in any keyspace group +''' + +["PD:keyspace:ErrKeyspaceNotInKeyspaceGroup"] +error = ''' +keyspace is not in this keyspace group +''' + +["PD:keyspace:ErrKeyspaceUpdateNotAllowed"] +error = ''' +cannot update keyspace that's %s +''' + +["PD:keyspace:ErrModifyDefaultKeyspace"] +error = ''' +cannot modify default keyspace's state +''' + +["PD:keyspacegroup:ErrDefaultKeyspaceGroupNotFound"] +error = ''' +default keyspace group not found +''' + +["PD:keyspacegroup:ErrKeyspaceGroupExists"] +error = ''' +keyspace group already exists +''' + +["PD:keyspacegroup:ErrKeyspaceGroupIDInvalid"] +error = ''' +the keyspace group id is invalid, %s +''' + +["PD:keyspacegroup:ErrKeyspaceGroupInMerging"] +error = ''' +keyspace group %v is in merging state +''' + +["PD:keyspacegroup:ErrKeyspaceGroupInSplit"] +error = ''' +keyspace group %v is in split state +''' + +["PD:keyspacegroup:ErrKeyspaceGroupIsMerging"] +error = ''' +the keyspace group %d is merging +''' + +["PD:keyspacegroup:ErrKeyspaceGroupNotEnoughReplicas"] +error = ''' +not enough replicas in the keyspace group +''' + +["PD:keyspacegroup:ErrKeyspaceGroupNotExists"] +error = ''' +keyspace group %v does not exist +''' + +["PD:keyspacegroup:ErrKeyspaceGroupNotInMerging"] +error = ''' +keyspace group %v is not in merging state +''' + +["PD:keyspacegroup:ErrKeyspaceGroupNotInSplit"] +error = ''' +keyspace group %v is not in split state +''' + +["PD:keyspacegroup:ErrKeyspaceGroupNotInitialized"] +error = ''' +the keyspace group %d isn't initialized +''' + +["PD:keyspacegroup:ErrKeyspaceGroupPrimaryNotFound"] +error = ''' +primary of keyspace group does not exist +''' + +["PD:keyspacegroup:ErrKeyspaceGroupWithEmptyKeyspace"] +error = ''' +keyspace group with empty keyspace +''' + +["PD:keyspacegroup:ErrLoadKeyspaceGroupsRetryExhausted"] +error = ''' +load keyspace groups retry exhausted, %s +''' + +["PD:keyspacegroup:ErrLoadKeyspaceGroupsTerminated"] +error = ''' +load keyspace groups terminated +''' + +["PD:keyspacegroup:ErrLoadKeyspaceGroupsTimeout"] +error = ''' +load keyspace groups timeout +''' + +["PD:keyspacegroup:ErrModifyDefaultKeyspaceGroup"] +error = ''' +default keyspace group cannot be modified +''' + +["PD:keyspacegroup:ErrNoAvailableNode"] +error = ''' +no available node for keyspace group +''' + +["PD:keyspacegroup:ErrNodeNotInKeyspaceGroup"] +error = ''' +the tso node is not in this keyspace group +''' + ["PD:leveldb:ErrLevelDBClose"] error = ''' close leveldb error @@ -531,21 +671,6 @@ error = ''' plugin is not found: %s ''' -["PD:operator:ErrRegionAbnormalPeer"] -error = ''' -region %v has abnormal peer -''' - -["PD:operator:ErrRegionNotAdjacent"] -error = ''' -two regions are not adjacent -''' - -["PD:operator:ErrRegionNotFound"] -error = ''' -region %v not found -''' - ["PD:os:ErrOSOpen"] error = ''' open error @@ -616,6 +741,21 @@ error = ''' failed to unmarshal proto ''' +["PD:region:ErrRegionAbnormalPeer"] +error = ''' +region %v has abnormal peer +''' + +["PD:region:ErrRegionNotAdjacent"] +error = ''' +two regions are not adjacent +''' + +["PD:region:ErrRegionNotFound"] +error = ''' +region %v not found +''' + ["PD:region:ErrRegionRuleContent"] error = ''' invalid region rule content, %s @@ -626,6 +766,21 @@ error = ''' region label rule not found for id %s ''' +["PD:region:ErrRegionSplitFailed"] +error = ''' +region split failed +''' + +["PD:region:ErrRegionSplitTimeout"] +error = ''' +region split timeout +''' + +["PD:region:ErrRegionStale"] +error = ''' +region is stale: region %v origin %v +''' + ["PD:resourcemanager:ErrDeleteReservedGroup"] error = ''' cannot delete reserved group @@ -726,24 +881,44 @@ error = ''' client url empty ''' +["PD:server:ErrClusterMismatched"] +error = ''' +rpc error: code = Unavailable desc = cluster mismatched +''' + ["PD:server:ErrConfiguration"] error = ''' cannot set invalid configuration ''' +["PD:server:ErrEtcdNotStarted"] +error = ''' +rpc error: code = Unavailable desc = server is started, but etcd not started +''' + ["PD:server:ErrLeaderNil"] error = ''' leader is nil ''' +["PD:server:ErrNotLeader"] +error = ''' +rpc error: code = Unavailable desc = not leader +''' + ["PD:server:ErrRateLimitExceeded"] error = ''' rate limit exceeded ''' +["PD:server:ErrSendHeartbeatTimeout"] +error = ''' +rpc error: code = DeadlineExceeded desc = send heartbeat timeout +''' + ["PD:server:ErrServerNotStarted"] error = ''' -server not started +rpc error: code = Unavailable desc = server not started ''' ["PD:server:ErrServiceRegistered"] @@ -771,6 +946,11 @@ error = ''' parse uint error ''' +["PD:tso:ErrForwardTSOTimeout"] +error = ''' +rpc error: code = DeadlineExceeded desc = forward tso request timeout +''' + ["PD:tso:ErrGenerateTimestamp"] error = ''' generate timestamp failed, %s @@ -796,44 +976,19 @@ error = ''' get min ts failed, %s ''' -["PD:tso:ErrKeyspaceGroupIDInvalid"] -error = ''' -the keyspace group id is invalid, %s -''' - -["PD:tso:ErrKeyspaceGroupIsMerging"] -error = ''' -the keyspace group %d is merging -''' - -["PD:tso:ErrKeyspaceGroupNotInitialized"] -error = ''' -the keyspace group %d isn't initialized -''' - -["PD:tso:ErrKeyspaceNotAssigned"] -error = ''' -the keyspace %d isn't assigned to any keyspace group -''' - -["PD:tso:ErrLoadKeyspaceGroupsRetryExhausted"] -error = ''' -load keyspace groups retry exhausted, %s -''' - -["PD:tso:ErrLoadKeyspaceGroupsTerminated"] +["PD:tso:ErrLogicOverflow"] error = ''' -load keyspace groups terminated +logic part overflow ''' -["PD:tso:ErrLoadKeyspaceGroupsTimeout"] +["PD:tso:ErrMaxCountTSOProxyRoutinesExceeded"] error = ''' -load keyspace groups timeout +rpc error: code = ResourceExhausted desc = max count of concurrent tso proxy routines exceeded ''' -["PD:tso:ErrLogicOverflow"] +["PD:tso:ErrNotFoundTSOAddr"] error = ''' -logic part overflow +rpc error: code = NotFound desc = not found tso address ''' ["PD:tso:ErrProxyTSOTimeout"] @@ -856,6 +1011,11 @@ error = ''' sync max ts failed, %s ''' +["PD:tso:ErrTSOProxyRecvFromClientTimeout"] +error = ''' +rpc error: code = DeadlineExceeded desc = tso proxy timeout when receiving from client; stream closed by server +''' + ["PD:tso:ErrUpdateTimestamp"] error = ''' update timestamp failed, %s diff --git a/pkg/core/region.go b/pkg/core/region.go index 57ba5cb3db0..94418de7323 100644 --- a/pkg/core/region.go +++ b/pkg/core/region.go @@ -27,7 +27,6 @@ import ( "github.com/docker/go-units" "github.com/gogo/protobuf/proto" - "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/kvproto/pkg/pdpb" "github.com/pingcap/kvproto/pkg/replication_modepb" @@ -41,11 +40,6 @@ import ( const randomRegionMaxRetry = 10 -// errRegionIsStale is error info for region is stale. -func errRegionIsStale(region *metapb.Region, origin *metapb.Region) error { - return errors.Errorf("region is stale: region %v origin %v", region, origin) -} - // RegionInfo records detail region info. // the properties are Read-Only once created except buckets. // the `buckets` could be modified by the request `report buckets` with greater version. @@ -889,7 +883,7 @@ func check(region, origin *RegionInfo, overlaps []*regionItem) error { for _, item := range overlaps { // PD ignores stale regions' heartbeats, unless it is recreated recently by unsafe recover operation. if region.GetRegionEpoch().GetVersion() < item.GetRegionEpoch().GetVersion() && !region.isRegionRecreated() { - return errRegionIsStale(region.GetMeta(), item.GetMeta()) + return errs.ErrRegionStale.FastGenByArgs(region.GetMeta(), item.GetMeta()) } } if origin == nil { @@ -902,7 +896,7 @@ func check(region, origin *RegionInfo, overlaps []*regionItem) error { isTermBehind := region.GetTerm() > 0 && region.GetTerm() < origin.GetTerm() // Region meta is stale, return an error. if (isTermBehind || r.GetVersion() < o.GetVersion() || r.GetConfVer() < o.GetConfVer()) && !region.isRegionRecreated() { - return errRegionIsStale(region.GetMeta(), origin.GetMeta()) + return errs.ErrRegionStale.FastGenByArgs(region.GetMeta(), origin.GetMeta()) } return nil diff --git a/pkg/errs/errno.go b/pkg/errs/errno.go index 9eedb144f95..5da18f80615 100644 --- a/pkg/errs/errno.go +++ b/pkg/errs/errno.go @@ -14,7 +14,11 @@ package errs -import "github.com/pingcap/errors" +import ( + "github.com/pingcap/errors" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" +) const ( // NotLeaderErr indicates the non-leader member received the requests which should be received by leader. @@ -40,24 +44,56 @@ var ( // tso errors var ( - ErrSetLocalTSOConfig = errors.Normalize("set local tso config failed, %s", errors.RFCCodeText("PD:tso:ErrSetLocalTSOConfig")) - ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) - ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) - ErrSyncMaxTS = errors.Normalize("sync max ts failed, %s", errors.RFCCodeText("PD:tso:ErrSyncMaxTS")) - ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) - ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) - ErrUpdateTimestamp = errors.Normalize("update timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrUpdateTimestamp")) - ErrLogicOverflow = errors.Normalize("logic part overflow", errors.RFCCodeText("PD:tso:ErrLogicOverflow")) - ErrProxyTSOTimeout = errors.Normalize("proxy tso timeout", errors.RFCCodeText("PD:tso:ErrProxyTSOTimeout")) - ErrKeyspaceGroupIDInvalid = errors.Normalize("the keyspace group id is invalid, %s", errors.RFCCodeText("PD:tso:ErrKeyspaceGroupIDInvalid")) - ErrGetAllocatorManager = errors.Normalize("get allocator manager failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocatorManager")) - ErrLoadKeyspaceGroupsTimeout = errors.Normalize("load keyspace groups timeout", errors.RFCCodeText("PD:tso:ErrLoadKeyspaceGroupsTimeout")) - ErrLoadKeyspaceGroupsTerminated = errors.Normalize("load keyspace groups terminated", errors.RFCCodeText("PD:tso:ErrLoadKeyspaceGroupsTerminated")) - ErrLoadKeyspaceGroupsRetryExhausted = errors.Normalize("load keyspace groups retry exhausted, %s", errors.RFCCodeText("PD:tso:ErrLoadKeyspaceGroupsRetryExhausted")) - ErrKeyspaceGroupNotInitialized = errors.Normalize("the keyspace group %d isn't initialized", errors.RFCCodeText("PD:tso:ErrKeyspaceGroupNotInitialized")) - ErrKeyspaceNotAssigned = errors.Normalize("the keyspace %d isn't assigned to any keyspace group", errors.RFCCodeText("PD:tso:ErrKeyspaceNotAssigned")) - ErrGetMinTS = errors.Normalize("get min ts failed, %s", errors.RFCCodeText("PD:tso:ErrGetMinTS")) - ErrKeyspaceGroupIsMerging = errors.Normalize("the keyspace group %d is merging", errors.RFCCodeText("PD:tso:ErrKeyspaceGroupIsMerging")) + ErrSetLocalTSOConfig = errors.Normalize("set local tso config failed, %s", errors.RFCCodeText("PD:tso:ErrSetLocalTSOConfig")) + ErrGetAllocator = errors.Normalize("get allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocator")) + ErrGetLocalAllocator = errors.Normalize("get local allocator failed, %s", errors.RFCCodeText("PD:tso:ErrGetLocalAllocator")) + ErrSyncMaxTS = errors.Normalize("sync max ts failed, %s", errors.RFCCodeText("PD:tso:ErrSyncMaxTS")) + ErrResetUserTimestamp = errors.Normalize("reset user timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrResetUserTimestamp")) + ErrGenerateTimestamp = errors.Normalize("generate timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrGenerateTimestamp")) + ErrUpdateTimestamp = errors.Normalize("update timestamp failed, %s", errors.RFCCodeText("PD:tso:ErrUpdateTimestamp")) + ErrLogicOverflow = errors.Normalize("logic part overflow", errors.RFCCodeText("PD:tso:ErrLogicOverflow")) + ErrProxyTSOTimeout = errors.Normalize("proxy tso timeout", errors.RFCCodeText("PD:tso:ErrProxyTSOTimeout")) + ErrGetAllocatorManager = errors.Normalize("get allocator manager failed, %s", errors.RFCCodeText("PD:tso:ErrGetAllocatorManager")) + ErrGetMinTS = errors.Normalize("get min ts failed, %s", errors.RFCCodeText("PD:tso:ErrGetMinTS")) + + ErrNotFoundTSOAddr = errors.Normalize(status.Errorf(codes.NotFound, "not found tso address").Error(), errors.RFCCodeText("PD:tso:ErrNotFoundTSOAddr")) + ErrForwardTSOTimeout = errors.Normalize(status.Errorf(codes.DeadlineExceeded, "forward tso request timeout").Error(), errors.RFCCodeText("PD:tso:ErrForwardTSOTimeout")) + ErrMaxCountTSOProxyRoutinesExceeded = errors.Normalize(status.Errorf(codes.ResourceExhausted, "max count of concurrent tso proxy routines exceeded").Error(), errors.RFCCodeText("PD:tso:ErrMaxCountTSOProxyRoutinesExceeded")) + ErrTSOProxyRecvFromClientTimeout = errors.Normalize(status.Errorf(codes.DeadlineExceeded, "tso proxy timeout when receiving from client; stream closed by server").Error(), errors.RFCCodeText("PD:tso:ErrTSOProxyRecvFromClientTimeout")) +) + +// keyspace errors +var ( + ErrKeyspaceNotFound = errors.Normalize("keyspace does not exist", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotFound")) + ErrKeyspaceExists = errors.Normalize("keyspace already exists", errors.RFCCodeText("PD:keyspace:ErrKeyspaceExists")) + ErrKeyspaceNotAssigned = errors.Normalize("the keyspace %d isn't assigned to any keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotAssigned")) + ErrModifyDefaultKeyspace = errors.Normalize("cannot modify default keyspace's state", errors.RFCCodeText("PD:keyspace:ErrModifyDefaultKeyspace")) + ErrKeyspaceNotInKeyspaceGroup = errors.Normalize("keyspace is not in this keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInKeyspaceGroup")) + ErrKeyspaceNotInAnyKeyspaceGroup = errors.Normalize("keyspace is not in any keyspace group", errors.RFCCodeText("PD:keyspace:ErrKeyspaceNotInAnyKeyspaceGroup")) + ErrKeyspaceUpdateNotAllowed = errors.Normalize("cannot update keyspace that's %s", errors.RFCCodeText("PD:keyspace:ErrKeyspaceUpdateNotAllowed")) +) + +// keyspace group errors +var ( + ErrKeyspaceGroupIDInvalid = errors.Normalize("the keyspace group id is invalid, %s", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupIDInvalid")) + ErrLoadKeyspaceGroupsTimeout = errors.Normalize("load keyspace groups timeout", errors.RFCCodeText("PD:keyspacegroup:ErrLoadKeyspaceGroupsTimeout")) + ErrLoadKeyspaceGroupsTerminated = errors.Normalize("load keyspace groups terminated", errors.RFCCodeText("PD:keyspacegroup:ErrLoadKeyspaceGroupsTerminated")) + ErrLoadKeyspaceGroupsRetryExhausted = errors.Normalize("load keyspace groups retry exhausted, %s", errors.RFCCodeText("PD:keyspacegroup:ErrLoadKeyspaceGroupsRetryExhausted")) + ErrKeyspaceGroupNotInitialized = errors.Normalize("the keyspace group %d isn't initialized", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupNotInitialized")) + ErrKeyspaceGroupIsMerging = errors.Normalize("the keyspace group %d is merging", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupIsMerging")) + ErrKeyspaceGroupExists = errors.Normalize("keyspace group already exists", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupExists")) + ErrKeyspaceGroupNotExists = errors.Normalize("keyspace group %v does not exist", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupNotExists")) + ErrKeyspaceGroupInSplit = errors.Normalize("keyspace group %v is in split state", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupInSplit")) + ErrKeyspaceGroupNotInSplit = errors.Normalize("keyspace group %v is not in split state", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupNotInSplit")) + ErrKeyspaceGroupInMerging = errors.Normalize("keyspace group %v is in merging state", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupInMerging")) + ErrKeyspaceGroupNotInMerging = errors.Normalize("keyspace group %v is not in merging state", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupNotInMerging")) + ErrNodeNotInKeyspaceGroup = errors.Normalize("the tso node is not in this keyspace group", errors.RFCCodeText("PD:keyspacegroup:ErrNodeNotInKeyspaceGroup")) + ErrNoAvailableNode = errors.Normalize("no available node for keyspace group", errors.RFCCodeText("PD:keyspacegroup:ErrNoAvailableNode")) + ErrKeyspaceGroupNotEnoughReplicas = errors.Normalize("not enough replicas in the keyspace group", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupNotEnoughReplicas")) + ErrKeyspaceGroupWithEmptyKeyspace = errors.Normalize("keyspace group with empty keyspace", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupWithEmptyKeyspace")) + ErrDefaultKeyspaceGroupNotFound = errors.Normalize("default keyspace group not found", errors.RFCCodeText("PD:keyspacegroup:ErrDefaultKeyspaceGroupNotFound")) + ErrModifyDefaultKeyspaceGroup = errors.Normalize("default keyspace group cannot be modified", errors.RFCCodeText("PD:keyspacegroup:ErrModifyDefaultKeyspaceGroup")) + ErrKeyspaceGroupPrimaryNotFound = errors.Normalize("primary of keyspace group does not exist", errors.RFCCodeText("PD:keyspacegroup:ErrKeyspaceGroupPrimaryNotFound")) ) // member errors @@ -94,20 +130,20 @@ var ( // operator errors var ( - // ErrOperatorNotFound is error info for operator not found. ErrOperatorNotFound = errors.Normalize("operator not found", errors.RFCCodeText("PD:operator:ErrOperatorNotFound")) - // ErrAddOperator is error info for already have an operator when adding operator. - ErrAddOperator = errors.Normalize("failed to add operator, maybe already have one", errors.RFCCodeText("PD:operator:ErrAddOperator")) + ErrAddOperator = errors.Normalize("failed to add operator, maybe already have one", errors.RFCCodeText("PD:operator:ErrAddOperator")) ) // region errors var ( - // ErrRegionNotAdjacent is error info for region not adjacent. - ErrRegionNotAdjacent = errors.Normalize("two regions are not adjacent", errors.RFCCodeText("PD:operator:ErrRegionNotAdjacent")) - // ErrRegionNotFound is error info for region not found. - ErrRegionNotFound = errors.Normalize("region %v not found", errors.RFCCodeText("PD:operator:ErrRegionNotFound")) - // ErrRegionAbnormalPeer is error info for region has abnormal peer. - ErrRegionAbnormalPeer = errors.Normalize("region %v has abnormal peer", errors.RFCCodeText("PD:operator:ErrRegionAbnormalPeer")) + ErrRegionNotAdjacent = errors.Normalize("two regions are not adjacent", errors.RFCCodeText("PD:region:ErrRegionNotAdjacent")) + ErrRegionNotFound = errors.Normalize("region %v not found", errors.RFCCodeText("PD:region:ErrRegionNotFound")) + ErrRegionAbnormalPeer = errors.Normalize("region %v has abnormal peer", errors.RFCCodeText("PD:region:ErrRegionAbnormalPeer")) + ErrRegionSplitTimeout = errors.Normalize("region split timeout", errors.RFCCodeText("PD:region:ErrRegionSplitTimeout")) + ErrRegionSplitFailed = errors.Normalize("region split failed", errors.RFCCodeText("PD:region:ErrRegionSplitFailed")) + ErrRegionRuleContent = errors.Normalize("invalid region rule content, %s", errors.RFCCodeText("PD:region:ErrRegionRuleContent")) + ErrRegionRuleNotFound = errors.Normalize("region label rule not found for id %s", errors.RFCCodeText("PD:region:ErrRegionRuleNotFound")) + ErrRegionStale = errors.Normalize("region is stale: region %v origin %v", errors.RFCCodeText("PD:region:ErrRegionStale")) ) // plugin errors @@ -159,12 +195,6 @@ var ( ErrBuildRuleList = errors.Normalize("build rule list failed, %s", errors.RFCCodeText("PD:placement:ErrBuildRuleList")) ) -// region label errors -var ( - ErrRegionRuleContent = errors.Normalize("invalid region rule content, %s", errors.RFCCodeText("PD:region:ErrRegionRuleContent")) - ErrRegionRuleNotFound = errors.Normalize("region label rule not found for id %s", errors.RFCCodeText("PD:region:ErrRegionRuleNotFound")) -) - // cluster errors var ( ErrNotBootstrapped = errors.Normalize("TiKV cluster not bootstrapped, please start TiKV first", errors.RFCCodeText("PD:cluster:ErrNotBootstrapped")) @@ -207,8 +237,12 @@ var ( ErrLeaderNil = errors.Normalize("leader is nil", errors.RFCCodeText("PD:server:ErrLeaderNil")) ErrCancelStartEtcd = errors.Normalize("etcd start canceled", errors.RFCCodeText("PD:server:ErrCancelStartEtcd")) ErrConfigItem = errors.Normalize("cannot set invalid configuration", errors.RFCCodeText("PD:server:ErrConfiguration")) - ErrServerNotStarted = errors.Normalize("server not started", errors.RFCCodeText("PD:server:ErrServerNotStarted")) + ErrServerNotStarted = errors.Normalize(status.Errorf(codes.Unavailable, "server not started").Error(), errors.RFCCodeText("PD:server:ErrServerNotStarted")) + ErrClusterMismatched = errors.Normalize(status.Errorf(codes.Unavailable, "cluster mismatched").Error(), errors.RFCCodeText("PD:server:ErrClusterMismatched")) ErrRateLimitExceeded = errors.Normalize("rate limit exceeded", errors.RFCCodeText("PD:server:ErrRateLimitExceeded")) + ErrNotLeader = errors.Normalize(status.Errorf(codes.Unavailable, "not leader").Error(), errors.RFCCodeText("PD:server:ErrNotLeader")) + ErrSendHeartbeatTimeout = errors.Normalize(status.Errorf(codes.DeadlineExceeded, "send heartbeat timeout").Error(), errors.RFCCodeText("PD:server:ErrSendHeartbeatTimeout")) + ErrEtcdNotStarted = errors.Normalize(status.Errorf(codes.Unavailable, "server is started, but etcd not started").Error(), errors.RFCCodeText("PD:server:ErrEtcdNotStarted")) ) // logutil errors @@ -246,23 +280,25 @@ var ( // etcd errors var ( - ErrNewEtcdClient = errors.Normalize("new etcd client failed", errors.RFCCodeText("PD:etcd:ErrNewEtcdClient")) - ErrStartEtcd = errors.Normalize("start etcd failed", errors.RFCCodeText("PD:etcd:ErrStartEtcd")) - ErrEtcdURLMap = errors.Normalize("etcd url map error", errors.RFCCodeText("PD:etcd:ErrEtcdURLMap")) - ErrEtcdGrantLease = errors.Normalize("etcd lease failed", errors.RFCCodeText("PD:etcd:ErrEtcdGrantLease")) - ErrEtcdTxnInternal = errors.Normalize("internal etcd transaction error occurred", errors.RFCCodeText("PD:etcd:ErrEtcdTxnInternal")) - ErrEtcdTxnConflict = errors.Normalize("etcd transaction failed, conflicted and rolled back", errors.RFCCodeText("PD:etcd:ErrEtcdTxnConflict")) - ErrEtcdKVPut = errors.Normalize("etcd KV put failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVPut")) - ErrEtcdKVDelete = errors.Normalize("etcd KV delete failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVDelete")) - ErrEtcdKVGet = errors.Normalize("etcd KV get failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVGet")) - ErrEtcdKVGetResponse = errors.Normalize("etcd invalid get value response %v, must only one", errors.RFCCodeText("PD:etcd:ErrEtcdKVGetResponse")) - ErrEtcdGetCluster = errors.Normalize("etcd get cluster from remote peer failed", errors.RFCCodeText("PD:etcd:ErrEtcdGetCluster")) - ErrEtcdMoveLeader = errors.Normalize("etcd move leader error", errors.RFCCodeText("PD:etcd:ErrEtcdMoveLeader")) - ErrEtcdTLSConfig = errors.Normalize("etcd TLS config error", errors.RFCCodeText("PD:etcd:ErrEtcdTLSConfig")) - ErrEtcdWatcherCancel = errors.Normalize("watcher canceled", errors.RFCCodeText("PD:etcd:ErrEtcdWatcherCancel")) - ErrCloseEtcdClient = errors.Normalize("close etcd client failed", errors.RFCCodeText("PD:etcd:ErrCloseEtcdClient")) - ErrEtcdMemberList = errors.Normalize("etcd member list failed", errors.RFCCodeText("PD:etcd:ErrEtcdMemberList")) - ErrEtcdMemberRemove = errors.Normalize("etcd remove member failed", errors.RFCCodeText("PD:etcd:ErrEtcdMemberRemove")) + ErrNewEtcdClient = errors.Normalize("new etcd client failed", errors.RFCCodeText("PD:etcd:ErrNewEtcdClient")) + ErrStartEtcd = errors.Normalize("start etcd failed", errors.RFCCodeText("PD:etcd:ErrStartEtcd")) + ErrEtcdURLMap = errors.Normalize("etcd url map error", errors.RFCCodeText("PD:etcd:ErrEtcdURLMap")) + ErrEtcdGrantLease = errors.Normalize("etcd lease failed", errors.RFCCodeText("PD:etcd:ErrEtcdGrantLease")) + ErrEtcdTxnInternal = errors.Normalize("internal etcd transaction error occurred", errors.RFCCodeText("PD:etcd:ErrEtcdTxnInternal")) + ErrEtcdTxnConflict = errors.Normalize("etcd transaction failed, conflicted and rolled back", errors.RFCCodeText("PD:etcd:ErrEtcdTxnConflict")) + ErrEtcdKVPut = errors.Normalize("etcd KV put failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVPut")) + ErrEtcdKVDelete = errors.Normalize("etcd KV delete failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVDelete")) + ErrEtcdKVGet = errors.Normalize("etcd KV get failed", errors.RFCCodeText("PD:etcd:ErrEtcdKVGet")) + ErrEtcdKVGetResponse = errors.Normalize("etcd invalid get value response %v, must only one", errors.RFCCodeText("PD:etcd:ErrEtcdKVGetResponse")) + ErrEtcdGetCluster = errors.Normalize("etcd get cluster from remote peer failed", errors.RFCCodeText("PD:etcd:ErrEtcdGetCluster")) + ErrEtcdMoveLeader = errors.Normalize("etcd move leader error", errors.RFCCodeText("PD:etcd:ErrEtcdMoveLeader")) + ErrEtcdTLSConfig = errors.Normalize("etcd TLS config error", errors.RFCCodeText("PD:etcd:ErrEtcdTLSConfig")) + ErrEtcdWatcherCancel = errors.Normalize("watcher canceled", errors.RFCCodeText("PD:etcd:ErrEtcdWatcherCancel")) + ErrCloseEtcdClient = errors.Normalize("close etcd client failed", errors.RFCCodeText("PD:etcd:ErrCloseEtcdClient")) + ErrEtcdMemberList = errors.Normalize("etcd member list failed", errors.RFCCodeText("PD:etcd:ErrEtcdMemberList")) + ErrEtcdMemberRemove = errors.Normalize("etcd remove member failed", errors.RFCCodeText("PD:etcd:ErrEtcdMemberRemove")) + ErrExceedMaxEtcdTxnOps = errors.Normalize("exceed max etcd txn operations", errors.RFCCodeText("PD:etcd:ErrExceedMaxEtcdTxnOps")) + ErrIllegalOperation = errors.Normalize("unknown operation", errors.RFCCodeText("PD:etcd:ErrIllegalOperation")) ) // dashboard errors diff --git a/pkg/gc/safepoint_v2.go b/pkg/gc/safepoint_v2.go index 665249bcab0..0e80aaf6f10 100644 --- a/pkg/gc/safepoint_v2.go +++ b/pkg/gc/safepoint_v2.go @@ -18,10 +18,10 @@ import ( "context" "time" - "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/keyspace" "github.com/tikv/pd/pkg/slice" "github.com/tikv/pd/pkg/storage/endpoint" @@ -99,11 +99,11 @@ func (manager *SafePointV2Manager) checkKeyspace(keyspaceID uint32, updateReques } // If a keyspace does not exist, then loading its gc safe point is prohibited. if meta == nil { - return keyspace.ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // If keyspace's state does not permit updating safe point, we return error. if updateRequest && !slice.Contains(allowUpdateSafePoint, meta.GetState()) { - return errors.Errorf("cannot update keyspace that's %s", meta.GetState().String()) + return errs.ErrKeyspaceUpdateNotAllowed.FastGenByArgs(meta.GetState().String()) } return nil }) diff --git a/pkg/keyspace/keyspace.go b/pkg/keyspace/keyspace.go index 1607676a37b..9862b7d1df3 100644 --- a/pkg/keyspace/keyspace.go +++ b/pkg/keyspace/keyspace.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/log" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/id" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/schedule/core" @@ -147,7 +148,7 @@ func (manager *Manager) Bootstrap() error { err = manager.saveNewKeyspace(defaultKeyspaceMeta) // It's possible that default keyspace already exists in the storage (e.g. PD restart/recover), // so we ignore the keyspaceExists error. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], defaultKeyspaceMeta.GetId(), opAdd); err != nil { @@ -168,7 +169,7 @@ func (manager *Manager) Bootstrap() error { } keyspace, err := manager.CreateKeyspace(req) // Ignore the keyspaceExists error for the same reason as saving default keyspace. - if err != nil && err != ErrKeyspaceExists { + if err != nil && err != errs.ErrKeyspaceExists { return err } if err := manager.kgm.UpdateKeyspaceForGroup(endpoint.Basic, config[TSOKeyspaceGroupIDKey], keyspace.GetId(), opAdd); err != nil { @@ -282,7 +283,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if nameExists { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } err = manager.store.SaveKeyspaceID(txn, keyspace.Id, keyspace.Name) if err != nil { @@ -295,7 +296,7 @@ func (manager *Manager) saveNewKeyspace(keyspace *keyspacepb.KeyspaceMeta) error return err } if loadedMeta != nil { - return ErrKeyspaceExists + return errs.ErrKeyspaceExists } return manager.store.SaveKeyspaceMeta(txn, keyspace) }) @@ -332,7 +333,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er ranges := keyspaceRule.Data.([]*labeler.KeyRangeRule) if len(ranges) < 2 { log.Warn("[keyspace] failed to split keyspace region with insufficient range", zap.Any("label-rule", keyspaceRule)) - return ErrRegionSplitFailed + return errs.ErrRegionSplitFailed } rawLeftBound, rawRightBound := ranges[0].StartKey, ranges[0].EndKey txnLeftBound, txnRightBound := ranges[1].StartKey, ranges[1].EndKey @@ -370,8 +371,7 @@ func (manager *Manager) splitKeyspaceRegion(id uint32, waitRegionSplit bool) (er zap.Uint32("keyspace-id", id), zap.Error(err), ) - err = ErrRegionSplitTimeout - return + return errs.ErrRegionSplitTimeout } log.Info("[keyspace] wait region split successfully", zap.Uint32("keyspace-id", id)) break @@ -396,14 +396,14 @@ func (manager *Manager) LoadKeyspace(name string) (*keyspacepb.KeyspaceMeta, err return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } meta, err = manager.store.LoadKeyspaceMeta(txn, id) if err != nil { return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -423,7 +423,7 @@ func (manager *Manager) LoadKeyspaceByID(spaceID uint32) (*keyspacepb.KeyspaceMe return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } return nil }) @@ -463,7 +463,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -473,7 +473,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Only keyspace with state listed in allowChangeConfig are allowed to change their config. if !slice.Contains(allowChangeConfig, meta.GetState()) { @@ -494,7 +494,7 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) case OpDel: delete(meta.Config, mutation.Key) default: - return errIllegalOperation + return errs.ErrIllegalOperation } } newConfig := meta.GetConfig() @@ -538,23 +538,21 @@ func (manager *Manager) UpdateKeyspaceConfig(name string, mutations []*Mutation) // UpdateKeyspaceState updates target keyspace to the given state if it's not already in that state. // It returns error if saving failed, operation not allowed, or if keyspace not exists. -func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.KeyspaceState, now int64) (*keyspacepb.KeyspaceMeta, error) { +func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.KeyspaceState, now int64) (meta *keyspacepb.KeyspaceMeta, err error) { // Changing the state of default keyspace is not allowed. if name == utils.DefaultKeyspaceName { - log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), - ) - return nil, ErrModifyDefaultKeyspace + err = errs.ErrModifyDefaultKeyspace + log.Warn("[keyspace] failed to update keyspace config", zap.Error(err)) + return nil, err } - var meta *keyspacepb.KeyspaceMeta - err := manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { + err = manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { // First get KeyspaceID from Name. loaded, id, err := manager.store.LoadKeyspaceID(txn, name) if err != nil { return err } if !loaded { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -564,7 +562,7 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -590,16 +588,13 @@ func (manager *Manager) UpdateKeyspaceState(name string, newState keyspacepb.Key // UpdateKeyspaceStateByID updates target keyspace to the given state if it's not already in that state. // It returns error if saving failed, operation not allowed, or if keyspace not exists. -func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.KeyspaceState, now int64) (*keyspacepb.KeyspaceMeta, error) { +func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.KeyspaceState, now int64) (meta *keyspacepb.KeyspaceMeta, err error) { // Changing the state of default keyspace is not allowed. if id == utils.DefaultKeyspaceID { - log.Warn("[keyspace] failed to update keyspace config", - zap.Error(ErrModifyDefaultKeyspace), - ) - return nil, ErrModifyDefaultKeyspace + err = errs.ErrModifyDefaultKeyspace + log.Warn("[keyspace] failed to update keyspace config", zap.Error(err)) + return nil, err } - var meta *keyspacepb.KeyspaceMeta - var err error err = manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { manager.metaLock.Lock(id) defer manager.metaLock.Unlock(id) @@ -609,7 +604,7 @@ func (manager *Manager) UpdateKeyspaceStateByID(id uint32, newState keyspacepb.K return err } if meta == nil { - return ErrKeyspaceNotFound + return errs.ErrKeyspaceNotFound } // Update keyspace meta. if err = updateKeyspaceState(meta, newState, now); err != nil { @@ -719,20 +714,19 @@ func (manager *Manager) PatrolKeyspaceAssignment(startKeyspaceID, endKeyspaceID }() for moreToPatrol { var defaultKeyspaceGroup *endpoint.KeyspaceGroup - err = manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) error { - var err error + err = manager.store.RunInTxn(manager.ctx, func(txn kv.Txn) (err error) { defaultKeyspaceGroup, err = manager.kgm.store.LoadKeyspaceGroup(txn, utils.DefaultKeyspaceGroupID) if err != nil { return err } if defaultKeyspaceGroup == nil { - return errors.Errorf("default keyspace group %d not found", utils.DefaultKeyspaceGroupID) + return errs.ErrDefaultKeyspaceGroupNotFound } if defaultKeyspaceGroup.IsSplitting() { - return ErrKeyspaceGroupInSplit(utils.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(utils.DefaultKeyspaceGroupID) } if defaultKeyspaceGroup.IsMerging() { - return ErrKeyspaceGroupInMerging(utils.DefaultKeyspaceGroupID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(utils.DefaultKeyspaceGroupID) } keyspaces, err := manager.store.LoadRangeKeyspace(txn, manager.nextPatrolStartID, MaxEtcdTxnOps) if err != nil { diff --git a/pkg/keyspace/tso_keyspace_group.go b/pkg/keyspace/tso_keyspace_group.go index a092e5b18a6..3d214c00c17 100644 --- a/pkg/keyspace/tso_keyspace_group.go +++ b/pkg/keyspace/tso_keyspace_group.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" "github.com/tikv/pd/pkg/balancer" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/discovery" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/slice" @@ -127,7 +128,7 @@ func (m *GroupManager) Bootstrap(ctx context.Context) error { // Ignore the error if default keyspace group already exists in the storage (e.g. PD restart/recover). err := m.saveKeyspaceGroups([]*endpoint.KeyspaceGroup{defaultKeyspaceGroup}, false) - if err != nil && err != ErrKeyspaceGroupExists { + if err != nil && err != errs.ErrKeyspaceGroupExists { return err } @@ -318,7 +319,7 @@ func (m *GroupManager) DeleteKeyspaceGroupByID(id uint32) (*endpoint.KeyspaceGro return nil } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } return m.store.DeleteKeyspaceGroup(txn, id) }); err != nil { @@ -344,13 +345,13 @@ func (m *GroupManager) saveKeyspaceGroups(keyspaceGroups []*endpoint.KeyspaceGro return err } if oldKG != nil && !overwrite { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } if oldKG.IsSplitting() && overwrite { - return ErrKeyspaceGroupInSplit(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(keyspaceGroup.ID) } if oldKG.IsMerging() && overwrite { - return ErrKeyspaceGroupInMerging(keyspaceGroup.ID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(keyspaceGroup.ID) } newKG := &endpoint.KeyspaceGroup{ ID: keyspaceGroup.ID, @@ -406,7 +407,7 @@ func (m *GroupManager) GetGroupByKeyspaceID(id uint32) (uint32, error) { } } } - return 0, ErrKeyspaceNotInAnyKeyspaceGroup + return 0, errs.ErrKeyspaceNotInAnyKeyspaceGroup } var failpointOnce sync.Once @@ -436,13 +437,13 @@ func (m *GroupManager) UpdateKeyspaceForGroup(userKind endpoint.UserKind, groupI func (m *GroupManager) updateKeyspaceForGroupLocked(userKind endpoint.UserKind, groupID uint64, keyspaceID uint32, mutation int) error { kg := m.groups[userKind].Get(uint32(groupID)) if kg == nil { - return ErrKeyspaceGroupNotExists(uint32(groupID)) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(uint32(groupID)) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(groupID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(groupID)) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(groupID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(groupID)) } changed := false @@ -496,13 +497,13 @@ func (m *GroupManager) UpdateKeyspaceGroup(oldGroupID, newGroupID string, oldUse return errors.Errorf("keyspace group %s not found in %s group", newGroupID, newUserKind) } if oldKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(oldID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(oldID)) } else if newKG.IsSplitting() { - return ErrKeyspaceGroupInSplit(uint32(newID)) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(uint32(newID)) } else if oldKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(oldID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(oldID)) } else if newKG.IsMerging() { - return ErrKeyspaceGroupInMerging(uint32(newID)) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(uint32(newID)) } var updateOld, updateNew bool @@ -548,15 +549,15 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitSourceID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitSourceID) } // A keyspace group can not take part in multiple split processes. if splitSourceKg.IsSplitting() { - return ErrKeyspaceGroupInSplit(splitSourceID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(splitSourceID) } // A keyspace group can not be split when it is in merging. if splitSourceKg.IsMerging() { - return ErrKeyspaceGroupInMerging(splitSourceID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(splitSourceID) } // Build the new keyspace groups for split source and target. var startKeyspaceID, endKeyspaceID uint32 @@ -570,7 +571,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( } // Check if the source keyspace group has enough replicas. if len(splitSourceKg.Members) < utils.DefaultKeyspaceGroupReplicaCount { - return ErrKeyspaceGroupNotEnoughReplicas + return errs.ErrKeyspaceGroupNotEnoughReplicas } // Check if the new keyspace group already exists. splitTargetKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetID) @@ -578,7 +579,7 @@ func (m *GroupManager) SplitKeyspaceGroupByID( return err } if splitTargetKg != nil { - return ErrKeyspaceGroupExists + return errs.ErrKeyspaceGroupExists } // Update the old keyspace group. splitSourceKg.Keyspaces = splitSourceKeyspaces @@ -619,7 +620,7 @@ func buildSplitKeyspaces( // Split according to the new keyspace list. if newNum != 0 { if newNum > oldNum { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( oldKeyspaceMap = make(map[uint32]struct{}, oldNum) @@ -630,10 +631,10 @@ func buildSplitKeyspaces( } for _, keyspace := range new { if keyspace == utils.DefaultKeyspaceID { - return nil, nil, ErrModifyDefaultKeyspace + return nil, nil, errs.ErrModifyDefaultKeyspace } if _, ok := oldKeyspaceMap[keyspace]; !ok { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } newKeyspaceMap[keyspace] = struct{}{} } @@ -658,7 +659,7 @@ func buildSplitKeyspaces( } // Split according to the start and end keyspace ID. if startKeyspaceID == 0 && endKeyspaceID == 0 { - return nil, nil, ErrKeyspaceNotInKeyspaceGroup + return nil, nil, errs.ErrKeyspaceNotInKeyspaceGroup } var ( newSplit = make([]uint32, 0, oldNum) @@ -677,7 +678,7 @@ func buildSplitKeyspaces( } // Check if the new keyspace list is empty. if len(newSplit) == 0 { - return nil, nil, ErrKeyspaceGroupWithEmptyKeyspace + return nil, nil, errs.ErrKeyspaceGroupWithEmptyKeyspace } // Get the split keyspace list for the old keyspace group. oldSplit := make([]uint32, 0, oldNum-len(newSplit)) @@ -701,11 +702,11 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitTargetKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetID) } // Check if it's in the split state. if !splitTargetKg.IsSplitTarget() { - return ErrKeyspaceGroupNotInSplit(splitTargetID) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetID) } // Load the split source keyspace group then. splitSourceKg, err = m.store.LoadKeyspaceGroup(txn, splitTargetKg.SplitSource()) @@ -713,10 +714,10 @@ func (m *GroupManager) FinishSplitKeyspaceByID(splitTargetID uint32) error { return err } if splitSourceKg == nil { - return ErrKeyspaceGroupNotExists(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(splitTargetKg.SplitSource()) } if !splitSourceKg.IsSplitSource() { - return ErrKeyspaceGroupNotInSplit(splitTargetKg.SplitSource()) + return errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(splitTargetKg.SplitSource()) } splitTargetKg.SplitState = nil splitSourceKg.SplitState = nil @@ -761,13 +762,13 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, desiredReplicaCount return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } exists := make(map[string]struct{}) for _, member := range kg.Members { @@ -785,11 +786,11 @@ func (m *GroupManager) AllocNodesForKeyspaceGroup(id uint32, desiredReplicaCount } countOfNodes := m.GetNodesCount() if countOfNodes < desiredReplicaCount || countOfNodes == 0 { // double check - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } addr := m.nodesBalancer.Next() if addr == "" { - return ErrNoAvailableNode + return errs.ErrNoAvailableNode } if _, ok := exists[addr]; ok { continue @@ -825,13 +826,13 @@ func (m *GroupManager) SetNodesForKeyspaceGroup(id uint32, nodes []string) error return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } members := make([]endpoint.KeyspaceGroupMember, 0, len(nodes)) for _, node := range nodes { @@ -862,13 +863,13 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior return err } if kg == nil { - return ErrKeyspaceGroupNotExists(id) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(id) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(id) } if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(id) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(id) } inKeyspaceGroup := false members := make([]endpoint.KeyspaceGroupMember, 0, len(kg.Members)) @@ -880,7 +881,7 @@ func (m *GroupManager) SetPriorityForKeyspaceGroup(id uint32, node string, prior members = append(members, member) } if !inKeyspaceGroup { - return ErrNodeNotInKeyspaceGroup + return errs.ErrNodeNotInKeyspaceGroup } kg.Members = members return m.store.SaveKeyspaceGroup(txn, kg) @@ -914,10 +915,10 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin // - Load and update the target keyspace group. // So we pre-check the number of operations to avoid exceeding the maximum number of etcd transaction. if (mergeListNum+1)*2 > MaxEtcdTxnOps { - return ErrExceedMaxEtcdTxnOps + return errs.ErrExceedMaxEtcdTxnOps } if slice.Contains(mergeList, utils.DefaultKeyspaceGroupID) { - return ErrModifyDefaultKeyspaceGroup + return errs.ErrModifyDefaultKeyspaceGroup } var ( groups = make(map[uint32]*endpoint.KeyspaceGroup, mergeListNum+1) @@ -933,15 +934,15 @@ func (m *GroupManager) MergeKeyspaceGroups(mergeTargetID uint32, mergeList []uin return err } if kg == nil { - return ErrKeyspaceGroupNotExists(kgID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(kgID) } // A keyspace group can not be merged if it's in splitting. if kg.IsSplitting() { - return ErrKeyspaceGroupInSplit(kgID) + return errs.ErrKeyspaceGroupInSplit.FastGenByArgs(kgID) } // A keyspace group can not be split when it is in merging. if kg.IsMerging() { - return ErrKeyspaceGroupInMerging(kgID) + return errs.ErrKeyspaceGroupInMerging.FastGenByArgs(kgID) } groups[kgID] = kg } @@ -1007,11 +1008,11 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if mergeTargetKg == nil { - return ErrKeyspaceGroupNotExists(mergeTargetID) + return errs.ErrKeyspaceGroupNotExists.FastGenByArgs(mergeTargetID) } // Check if it's in the merging state. if !mergeTargetKg.IsMergeTarget() { - return ErrKeyspaceGroupNotInMerging(mergeTargetID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(mergeTargetID) } // Make sure all merging keyspace groups are deleted. for _, kgID := range mergeTargetKg.MergeState.MergeList { @@ -1020,7 +1021,7 @@ func (m *GroupManager) FinishMergeKeyspaceByID(mergeTargetID uint32) error { return err } if kg != nil { - return ErrKeyspaceGroupNotInMerging(kgID) + return errs.ErrKeyspaceGroupNotInMerging.FastGenByArgs(kgID) } } mergeList = mergeTargetKg.MergeState.MergeList @@ -1144,7 +1145,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if kg == nil { - return "", ErrKeyspaceGroupNotExists(id) + return "", errs.ErrKeyspaceGroupNotExists.FastGenByArgs(id) } rootPath := endpoint.TSOSvcRootPath(m.clusterID) @@ -1155,7 +1156,7 @@ func (m *GroupManager) GetKeyspaceGroupPrimaryByID(id uint32) (string, error) { return "", err } if !ok { - return "", ErrKeyspaceGroupPrimaryNotFound + return "", errs.ErrKeyspaceGroupPrimaryNotFound } // The format of leader name is address-groupID. contents := strings.Split(leader.GetName(), "-") diff --git a/pkg/keyspace/tso_keyspace_group_test.go b/pkg/keyspace/tso_keyspace_group_test.go index 993923d2fd7..c05c6d122a4 100644 --- a/pkg/keyspace/tso_keyspace_group_test.go +++ b/pkg/keyspace/tso_keyspace_group_test.go @@ -22,6 +22,7 @@ import ( "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/utils" "github.com/tikv/pd/pkg/mock/mockcluster" "github.com/tikv/pd/pkg/mock/mockconfig" @@ -253,13 +254,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { re.NoError(err) // split the default keyspace err = suite.kgm.SplitKeyspaceGroupByID(0, 4, []uint32{utils.DefaultKeyspaceID}) - re.ErrorIs(err, ErrModifyDefaultKeyspace) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspace) // split the keyspace group 1 to 4 err = suite.kgm.SplitKeyspaceGroupByID(1, 4, []uint32{444}) - re.ErrorIs(err, ErrKeyspaceGroupNotEnoughReplicas) + re.ErrorIs(err, errs.ErrKeyspaceGroupNotEnoughReplicas) // split the keyspace group 2 to 4 without giving any keyspace err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) // split the keyspace group 2 to 4 err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) re.NoError(err) @@ -280,25 +281,25 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // finish the split of the keyspace group 2 err = suite.kgm.FinishSplitKeyspaceByID(2) - re.ErrorContains(err, ErrKeyspaceGroupNotInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotInSplit.FastGenByArgs(2).Error()) // finish the split of a non-existing keyspace group err = suite.kgm.FinishSplitKeyspaceByID(5) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // split the in-split keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{333}) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) // remove the in-split keyspace group kg2, err = suite.kgm.DeleteKeyspaceGroupByID(2) re.Nil(kg2) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) kg4, err = suite.kgm.DeleteKeyspaceGroupByID(4) re.Nil(kg4) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // update the in-split keyspace group err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "2", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(2).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(2).Error()) err = suite.kg.kgm.UpdateKeyspaceForGroup(endpoint.Standard, "4", 444, opAdd) - re.ErrorContains(err, ErrKeyspaceGroupInSplit(4).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupInSplit.FastGenByArgs(4).Error()) // finish the split of keyspace group 4 err = suite.kgm.FinishSplitKeyspaceByID(4) @@ -318,13 +319,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplit() { // split a non-existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(3, 5, nil) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(3).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(3).Error()) // split into an existing keyspace group err = suite.kgm.SplitKeyspaceGroupByID(2, 4, []uint32{111}) - re.ErrorIs(err, ErrKeyspaceGroupExists) + re.ErrorIs(err, errs.ErrKeyspaceGroupExists) // split with the wrong keyspaces. err = suite.kgm.SplitKeyspaceGroupByID(2, 5, []uint32{111, 222, 444}) - re.ErrorIs(err, ErrKeyspaceNotInKeyspaceGroup) + re.ErrorIs(err, errs.ErrKeyspaceNotInKeyspaceGroup) } func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupSplitRange() { @@ -446,13 +447,13 @@ func (suite *keyspaceGroupTestSuite) TestKeyspaceGroupMerge() { // merge a non-existing keyspace group err = suite.kgm.MergeKeyspaceGroups(4, []uint32{5}) - re.ErrorContains(err, ErrKeyspaceGroupNotExists(5).Error()) + re.ErrorContains(err, errs.ErrKeyspaceGroupNotExists.FastGenByArgs(5).Error()) // merge with the number of keyspace groups exceeds the limit err = suite.kgm.MergeKeyspaceGroups(1, make([]uint32, MaxEtcdTxnOps/2)) - re.ErrorIs(err, ErrExceedMaxEtcdTxnOps) + re.ErrorIs(err, errs.ErrExceedMaxEtcdTxnOps) // merge the default keyspace group err = suite.kgm.MergeKeyspaceGroups(1, []uint32{utils.DefaultKeyspaceGroupID}) - re.ErrorIs(err, ErrModifyDefaultKeyspaceGroup) + re.ErrorIs(err, errs.ErrModifyDefaultKeyspaceGroup) } func TestBuildSplitKeyspaces(t *testing.T) { @@ -481,7 +482,7 @@ func TestBuildSplitKeyspaces(t *testing.T) { { old: []uint32{1, 2, 3, 4, 5}, new: []uint32{6}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, { old: []uint32{1, 2}, @@ -542,11 +543,11 @@ func TestBuildSplitKeyspaces(t *testing.T) { old: []uint32{1, 2, 3, 4, 5}, startKeyspaceID: 7, endKeyspaceID: 10, - err: ErrKeyspaceGroupWithEmptyKeyspace, + err: errs.ErrKeyspaceGroupWithEmptyKeyspace, }, { old: []uint32{1, 2, 3, 4, 5}, - err: ErrKeyspaceNotInKeyspaceGroup, + err: errs.ErrKeyspaceNotInKeyspaceGroup, }, } for idx, testCase := range testCases { diff --git a/pkg/keyspace/util.go b/pkg/keyspace/util.go index aa8d0f350ea..03db89abb58 100644 --- a/pkg/keyspace/util.go +++ b/pkg/keyspace/util.go @@ -37,57 +37,6 @@ const ( ) var ( - // ErrKeyspaceNotFound is used to indicate target keyspace does not exist. - ErrKeyspaceNotFound = errors.New("keyspace does not exist") - // ErrRegionSplitTimeout indices to split region timeout - ErrRegionSplitTimeout = errors.New("region split timeout") - // ErrRegionSplitFailed indices to split region failed - ErrRegionSplitFailed = errors.New("region split failed") - // ErrKeyspaceExists indicates target keyspace already exists. - // It's used when creating a new keyspace. - ErrKeyspaceExists = errors.New("keyspace already exists") - // ErrKeyspaceGroupExists indicates target keyspace group already exists. - ErrKeyspaceGroupExists = errors.New("keyspace group already exists") - // ErrKeyspaceGroupNotExists is used to indicate target keyspace group does not exist. - ErrKeyspaceGroupNotExists = func(groupID uint32) error { - return errors.Errorf("keyspace group %v does not exist", groupID) - } - // ErrKeyspaceGroupInSplit is used to indicate target keyspace group is in split state. - ErrKeyspaceGroupInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in split state", groupID) - } - // ErrKeyspaceGroupNotInSplit is used to indicate target keyspace group is not in split state. - ErrKeyspaceGroupNotInSplit = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in split state", groupID) - } - // ErrKeyspaceGroupInMerging is used to indicate target keyspace group is in merging state. - ErrKeyspaceGroupInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is in merging state", groupID) - } - // ErrKeyspaceGroupNotInMerging is used to indicate target keyspace group is not in merging state. - ErrKeyspaceGroupNotInMerging = func(groupID uint32) error { - return errors.Errorf("keyspace group %v is not in merging state", groupID) - } - // ErrKeyspaceNotInKeyspaceGroup is used to indicate target keyspace is not in this keyspace group. - ErrKeyspaceNotInKeyspaceGroup = errors.New("keyspace is not in this keyspace group") - // ErrKeyspaceNotInAnyKeyspaceGroup is used to indicate target keyspace is not in any keyspace group. - ErrKeyspaceNotInAnyKeyspaceGroup = errors.New("keyspace is not in any keyspace group") - // ErrNodeNotInKeyspaceGroup is used to indicate the tso node is not in this keyspace group. - ErrNodeNotInKeyspaceGroup = errors.New("the tso node is not in this keyspace group") - // ErrKeyspaceGroupNotEnoughReplicas is used to indicate not enough replicas in the keyspace group. - ErrKeyspaceGroupNotEnoughReplicas = errors.New("not enough replicas in the keyspace group") - // ErrKeyspaceGroupWithEmptyKeyspace is used to indicate keyspace group with empty keyspace. - ErrKeyspaceGroupWithEmptyKeyspace = errors.New("keyspace group with empty keyspace") - // ErrModifyDefaultKeyspaceGroup is used to indicate that default keyspace group cannot be modified. - ErrModifyDefaultKeyspaceGroup = errors.New("default keyspace group cannot be modified") - // ErrNoAvailableNode is used to indicate no available node in the keyspace group. - ErrNoAvailableNode = errors.New("no available node") - // ErrExceedMaxEtcdTxnOps is used to indicate the number of etcd txn operations exceeds the limit. - ErrExceedMaxEtcdTxnOps = errors.New("exceed max etcd txn operations") - // ErrModifyDefaultKeyspace is used to indicate that default keyspace cannot be modified. - ErrModifyDefaultKeyspace = errors.New("cannot modify default keyspace's state") - errIllegalOperation = errors.New("unknown operation") - // stateTransitionTable lists all allowed next state for the given current state. // Note that transit from any state to itself is allowed for idempotence. stateTransitionTable = map[keyspacepb.KeyspaceState][]keyspacepb.KeyspaceState{ @@ -98,9 +47,6 @@ var ( } // Only keyspaces in the state specified by allowChangeConfig are allowed to change their config. allowChangeConfig = []keyspacepb.KeyspaceState{keyspacepb.KeyspaceState_ENABLED, keyspacepb.KeyspaceState_DISABLED} - - // ErrKeyspaceGroupPrimaryNotFound is used to indicate primary of target keyspace group does not exist. - ErrKeyspaceGroupPrimaryNotFound = errors.New("primary of keyspace group does not exist") ) // validateID check if keyspace falls within the acceptable range. diff --git a/pkg/mcs/scheduling/server/grpc_service.go b/pkg/mcs/scheduling/server/grpc_service.go index f615e0c37c0..b85e2ded650 100644 --- a/pkg/mcs/scheduling/server/grpc_service.go +++ b/pkg/mcs/scheduling/server/grpc_service.go @@ -25,14 +25,6 @@ import ( "github.com/tikv/pd/pkg/utils/apiutil" "go.uber.org/zap" "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" -) - -// gRPC errors -var ( - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrClusterMismatched = status.Errorf(codes.Unavailable, "cluster mismatched") ) // SetUpRestHandler is a hook to sets up the REST service. diff --git a/pkg/mcs/tso/server/grpc_service.go b/pkg/mcs/tso/server/grpc_service.go index 40a308c72f8..14bd09816f6 100644 --- a/pkg/mcs/tso/server/grpc_service.go +++ b/pkg/mcs/tso/server/grpc_service.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/kvproto/pkg/tsopb" "github.com/pingcap/log" bs "github.com/tikv/pd/pkg/basicserver" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/mcs/registry" "github.com/tikv/pd/pkg/utils/apiutil" "github.com/tikv/pd/pkg/utils/grpcutil" @@ -35,12 +36,6 @@ import ( "google.golang.org/grpc/status" ) -// gRPC errors -var ( - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrClusterMismatched = status.Errorf(codes.Unavailable, "cluster mismatched") -) - var _ tsopb.TSOServer = (*Service)(nil) // SetUpRestHandler is a hook to sets up the REST service. @@ -134,7 +129,7 @@ func (s *Service) Tso(stream tsopb.TSO_TsoServer) error { start := time.Now() // TSO uses leader lease to determine validity. No need to check leader here. if s.IsClosed() { - return status.Errorf(codes.Unknown, "server not started") + return errs.ErrServerNotStarted } header := request.GetHeader() clusterID := header.GetClusterId() @@ -253,10 +248,10 @@ func (s *Service) GetMinTS( func (s *Service) validRequest(header *tsopb.RequestHeader) (tsopb.ErrorType, error) { if s.IsClosed() || s.keyspaceGroupManager == nil { - return tsopb.ErrorType_NOT_BOOTSTRAPPED, ErrNotStarted + return tsopb.ErrorType_NOT_BOOTSTRAPPED, errs.ErrServerNotStarted } if header == nil || header.GetClusterId() != s.clusterID { - return tsopb.ErrorType_CLUSTER_MISMATCHED, ErrClusterMismatched + return tsopb.ErrorType_CLUSTER_MISMATCHED, errs.ErrClusterMismatched } return tsopb.ErrorType_OK, nil } diff --git a/pkg/mcs/tso/server/server.go b/pkg/mcs/tso/server/server.go index 40958ca463c..9de1ad939a5 100644 --- a/pkg/mcs/tso/server/server.go +++ b/pkg/mcs/tso/server/server.go @@ -277,7 +277,7 @@ func (s *Server) IsLocalRequest(forwardedHost string) bool { // TODO: Check if the sender is from the global TSO allocator func (s *Server) ValidateInternalRequest(_ *tsopb.RequestHeader, _ bool) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrServerNotStarted } return nil } @@ -286,7 +286,7 @@ func (s *Server) ValidateInternalRequest(_ *tsopb.RequestHeader, _ bool) error { // TODO: Check if the keyspace replica is the primary func (s *Server) ValidateRequest(header *tsopb.RequestHeader) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrServerNotStarted } if header.GetClusterId() != s.clusterID { return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", s.clusterID, header.GetClusterId()) diff --git a/server/grpc_service.go b/server/grpc_service.go index 5a483b71818..8060daa9ec0 100644 --- a/server/grpc_service.go +++ b/server/grpc_service.go @@ -60,20 +60,6 @@ const ( defaultGRPCDialTimeout = 3 * time.Second ) -// gRPC errors -var ( - // ErrNotLeader is returned when current server is not the leader and not possible to process request. - // TODO: work as proxy. - ErrNotLeader = status.Errorf(codes.Unavailable, "not leader") - ErrNotStarted = status.Errorf(codes.Unavailable, "server not started") - ErrSendHeartbeatTimeout = status.Errorf(codes.DeadlineExceeded, "send heartbeat timeout") - ErrNotFoundTSOAddr = status.Errorf(codes.NotFound, "not found tso address") - ErrForwardTSOTimeout = status.Errorf(codes.DeadlineExceeded, "forward tso request timeout") - ErrMaxCountTSOProxyRoutinesExceeded = status.Errorf(codes.ResourceExhausted, "max count of concurrent tso proxy routines exceeded") - ErrTSOProxyRecvFromClientTimeout = status.Errorf(codes.DeadlineExceeded, "tso proxy timeout when receiving from client; stream closed by server") - ErrEtcdNotStarted = status.Errorf(codes.Unavailable, "server is started, but etcd not started") -) - // GrpcServer wraps Server to provide grpc service. type GrpcServer struct { *Server @@ -413,7 +399,7 @@ func (s *GrpcServer) Tso(stream pdpb.PD_TsoServer) error { start := time.Now() // TSO uses leader lease to determine validity. No need to check leader here. if s.IsClosed() { - return status.Errorf(codes.Unknown, "server not started") + return errs.ErrServerNotStarted } if request.GetHeader().GetClusterId() != s.clusterID { return status.Errorf(codes.FailedPrecondition, @@ -457,7 +443,7 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { maxConcurrentTSOProxyStreamings := int32(s.GetMaxConcurrentTSOProxyStreamings()) if maxConcurrentTSOProxyStreamings >= 0 { if newCount := s.concurrentTSOProxyStreamings.Add(1); newCount > maxConcurrentTSOProxyStreamings { - return errors.WithStack(ErrMaxCountTSOProxyRoutinesExceeded) + return errors.WithStack(errs.ErrMaxCountTSOProxyRoutinesExceeded) } } @@ -487,7 +473,7 @@ func (s *GrpcServer) forwardTSO(stream pdpb.PD_TsoServer) error { forwardedHost, ok := s.GetServicePrimaryAddr(stream.Context(), utils.TSOServiceName) if !ok || len(forwardedHost) == 0 { - return errors.WithStack(ErrNotFoundTSOAddr) + return errors.WithStack(errs.ErrNotFoundTSOAddr) } if forwardStream == nil || lastForwardedHost != forwardedHost { if cancelForward != nil { @@ -653,7 +639,7 @@ func (s *tsoServer) Send(m *pdpb.TsoResponse) error { return errors.WithStack(err) case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return ErrForwardTSOTimeout + return errs.ErrForwardTSOTimeout } } @@ -683,7 +669,7 @@ func (s *tsoServer) Recv(timeout time.Duration) (*pdpb.TsoRequest, error) { return req.request, nil case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return nil, ErrTSOProxyRecvFromClientTimeout + return nil, errs.ErrTSOProxyRecvFromClientTimeout } } @@ -692,7 +678,7 @@ func (s *GrpcServer) getForwardedHost(ctx, streamCtx context.Context) (forwarded var ok bool forwardedHost, ok = s.GetServicePrimaryAddr(ctx, utils.TSOServiceName) if !ok || len(forwardedHost) == 0 { - return "", ErrNotFoundTSOAddr + return "", errs.ErrNotFoundTSOAddr } } else if fh := grpcutil.GetForwardedHost(streamCtx); !s.isLocalRequest(fh) { forwardedHost = fh @@ -1070,7 +1056,7 @@ func (b *bucketHeartbeatServer) Send(bucket *pdpb.ReportBucketsResponse) error { return err case <-timer.C: atomic.StoreInt32(&b.closed, 1) - return ErrSendHeartbeatTimeout + return errs.ErrSendHeartbeatTimeout } } @@ -1112,7 +1098,7 @@ func (s *heartbeatServer) Send(m *pdpb.RegionHeartbeatResponse) error { return errors.WithStack(err) case <-timer.C: atomic.StoreInt32(&s.closed, 1) - return ErrSendHeartbeatTimeout + return errs.ErrSendHeartbeatTimeout } } @@ -1805,11 +1791,11 @@ func (s *GrpcServer) GetGCSafePoint(ctx context.Context, request *pdpb.GetGCSafe // SyncRegions syncs the regions. func (s *GrpcServer) SyncRegions(stream pdpb.PD_SyncRegionsServer) error { if s.IsClosed() || s.cluster == nil { - return ErrNotStarted + return errs.ErrServerNotStarted } ctx := s.cluster.Context() if ctx == nil { - return ErrNotStarted + return errs.ErrServerNotStarted } return s.cluster.GetRegionSyncer().Sync(ctx, stream) } @@ -1945,7 +1931,7 @@ func (s *GrpcServer) GetOperator(ctx context.Context, request *pdpb.GetOperatorR // TODO: Call it in gRPC interceptor. func (s *GrpcServer) validateRequest(header *pdpb.RequestHeader) error { if s.IsClosed() || !s.member.IsLeader() { - return ErrNotLeader + return errs.ErrNotLeader } if header.GetClusterId() != s.clusterID { return status.Errorf(codes.FailedPrecondition, "mismatch cluster id, need %d but got %d", s.clusterID, header.GetClusterId()) @@ -2178,7 +2164,7 @@ func (s *GrpcServer) GetDCLocationInfo(ctx context.Context, request *pdpb.GetDCL return nil, err } if !s.member.IsLeader() { - return nil, ErrNotLeader + return nil, errs.ErrNotLeader } am := s.GetTSOAllocatorManager() info, ok := am.GetDCLocationInfo(request.GetDcLocation()) @@ -2213,7 +2199,7 @@ func (s *GrpcServer) GetDCLocationInfo(ctx context.Context, request *pdpb.GetDCL // the gRPC communication between PD servers internally. func (s *GrpcServer) validateInternalRequest(header *pdpb.RequestHeader, onlyAllowLeader bool) error { if s.IsClosed() { - return ErrNotStarted + return errs.ErrServerNotStarted } // If onlyAllowLeader is true, check whether the sender is PD leader. if onlyAllowLeader { @@ -2348,7 +2334,7 @@ func (s *GrpcServer) getGlobalTSOFromTSOServer(ctx context.Context) (pdpb.Timest for i := 0; i < maxRetryTimesRequestTSOServer; i++ { forwardedHost, ok := s.GetServicePrimaryAddr(ctx, utils.TSOServiceName) if !ok || forwardedHost == "" { - return pdpb.Timestamp{}, ErrNotFoundTSOAddr + return pdpb.Timestamp{}, errs.ErrNotFoundTSOAddr } forwardStream, err = s.getTSOForwardStream(forwardedHost) if err != nil { @@ -2420,7 +2406,7 @@ const globalConfigPath = "/global/config/" // it should be set to `Payload bytes` instead of `Value string` func (s *GrpcServer) StoreGlobalConfig(_ context.Context, request *pdpb.StoreGlobalConfigRequest) (*pdpb.StoreGlobalConfigResponse, error) { if s.client == nil { - return nil, ErrEtcdNotStarted + return nil, errs.ErrEtcdNotStarted } configPath := request.GetConfigPath() if configPath == "" { @@ -2457,7 +2443,7 @@ func (s *GrpcServer) StoreGlobalConfig(_ context.Context, request *pdpb.StoreGlo // - `ConfigPath` if `Names` is nil can get all values and revision of current path func (s *GrpcServer) LoadGlobalConfig(ctx context.Context, request *pdpb.LoadGlobalConfigRequest) (*pdpb.LoadGlobalConfigResponse, error) { if s.client == nil { - return nil, ErrEtcdNotStarted + return nil, errs.ErrEtcdNotStarted } configPath := request.GetConfigPath() if configPath == "" { @@ -2496,7 +2482,7 @@ func (s *GrpcServer) LoadGlobalConfig(ctx context.Context, request *pdpb.LoadGlo // Watch on revision which greater than or equal to the required revision. func (s *GrpcServer) WatchGlobalConfig(req *pdpb.WatchGlobalConfigRequest, server pdpb.PD_WatchGlobalConfigServer) error { if s.client == nil { - return ErrEtcdNotStarted + return errs.ErrEtcdNotStarted } ctx, cancel := context.WithCancel(s.Context()) defer cancel() diff --git a/server/keyspace_service.go b/server/keyspace_service.go index b17239ba0a4..0091bcf40a9 100644 --- a/server/keyspace_service.go +++ b/server/keyspace_service.go @@ -22,7 +22,7 @@ import ( "github.com/gogo/protobuf/proto" "github.com/pingcap/kvproto/pkg/keyspacepb" "github.com/pingcap/kvproto/pkg/pdpb" - "github.com/tikv/pd/pkg/keyspace" + "github.com/tikv/pd/pkg/errs" "github.com/tikv/pd/pkg/storage/endpoint" "github.com/tikv/pd/pkg/utils/etcdutil" "go.etcd.io/etcd/clientv3" @@ -37,9 +37,9 @@ type KeyspaceServer struct { // getErrorHeader returns corresponding ResponseHeader based on err. func (s *KeyspaceServer) getErrorHeader(err error) *pdpb.ResponseHeader { switch err { - case keyspace.ErrKeyspaceExists: + case errs.ErrKeyspaceExists: return s.wrapErrorToHeader(pdpb.ErrorType_DUPLICATED_ENTRY, err.Error()) - case keyspace.ErrKeyspaceNotFound: + case errs.ErrKeyspaceNotFound: return s.wrapErrorToHeader(pdpb.ErrorType_ENTRY_NOT_FOUND, err.Error()) default: return s.wrapErrorToHeader(pdpb.ErrorType_UNKNOWN, err.Error())