Replace error code with merr for RateLimiter (#27176)

Signed-off-by: yah01 <yah2er0ne@outlook.com>
pull/27192/head
yah01 2023-09-19 10:37:23 +08:00 committed by GitHub
parent fc694bd56d
commit b9ab18d692
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 154 additions and 94 deletions

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus/pkg/config"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
@ -68,37 +69,37 @@ func NewMultiRateLimiter() *MultiRateLimiter {
}
// Check checks if request would be limited or denied.
func (m *MultiRateLimiter) Check(collectionID int64, rt internalpb.RateType, n int) commonpb.ErrorCode {
func (m *MultiRateLimiter) Check(collectionID int64, rt internalpb.RateType, n int) error {
if !Params.QuotaConfig.QuotaAndLimitsEnabled.GetAsBool() {
return commonpb.ErrorCode_Success
return nil
}
m.quotaStatesMu.RLock()
defer m.quotaStatesMu.RUnlock()
checkFunc := func(limiter *rateLimiter) commonpb.ErrorCode {
checkFunc := func(limiter *rateLimiter) error {
if limiter == nil {
return commonpb.ErrorCode_Success
return nil
}
limit, rate := limiter.limit(rt, n)
if rate == 0 {
return limiter.getErrorCode(rt)
return limiter.getError(rt)
}
if limit {
return commonpb.ErrorCode_RateLimit
return merr.WrapErrServiceRateLimit(rate)
}
return commonpb.ErrorCode_Success
return nil
}
// first, check global level rate limits
ret := checkFunc(m.globalDDLLimiter)
// second check collection level rate limits
if ret == commonpb.ErrorCode_Success && !IsDDLRequest(rt) {
if ret == nil && !IsDDLRequest(rt) {
// only dml and dql have collection level rate limits
ret = checkFunc(m.collectionLimiters[collectionID])
if ret != commonpb.ErrorCode_Success {
if ret != nil {
m.globalDDLLimiter.cancel(rt, n)
}
}
@ -237,18 +238,18 @@ func (rl *rateLimiter) setRates(collectionRate *proxypb.CollectionRate) error {
return nil
}
func (rl *rateLimiter) getErrorCode(rt internalpb.RateType) commonpb.ErrorCode {
func (rl *rateLimiter) getError(rt internalpb.RateType) error {
switch rt {
case internalpb.RateType_DMLInsert, internalpb.RateType_DMLUpsert, internalpb.RateType_DMLDelete, internalpb.RateType_DMLBulkLoad:
if errCode, ok := rl.quotaStates.Get(milvuspb.QuotaState_DenyToWrite); ok {
return errCode
return merr.OldCodeToMerr(errCode)
}
case internalpb.RateType_DQLSearch, internalpb.RateType_DQLQuery:
if errCode, ok := rl.quotaStates.Get(milvuspb.QuotaState_DenyToRead); ok {
return errCode
return merr.OldCodeToMerr(errCode)
}
}
return commonpb.ErrorCode_Success
return nil
}
// setRateGaugeByRateType sets ProxyLimiterRate metrics.

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/proxypb"
"github.com/milvus-io/milvus/pkg/util/etcd"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
"github.com/stretchr/testify/assert"
@ -50,19 +51,19 @@ func TestMultiRateLimiter(t *testing.T) {
}
for _, rt := range internalpb.RateType_value {
if IsDDLRequest(internalpb.RateType(rt)) {
errCode := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(collectionID, internalpb.RateType(rt), 5)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(collectionID, internalpb.RateType(rt), 5)
assert.Equal(t, commonpb.ErrorCode_RateLimit, errCode)
err := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.NoError(t, err)
err = multiLimiter.Check(collectionID, internalpb.RateType(rt), 5)
assert.NoError(t, err)
err = multiLimiter.Check(collectionID, internalpb.RateType(rt), 5)
assert.ErrorIs(t, err, merr.ErrServiceRateLimit)
} else {
errCode := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(collectionID, internalpb.RateType(rt), math.MaxInt)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(collectionID, internalpb.RateType(rt), math.MaxInt)
assert.Equal(t, commonpb.ErrorCode_RateLimit, errCode)
err := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.NoError(t, err)
err = multiLimiter.Check(collectionID, internalpb.RateType(rt), math.MaxInt)
assert.NoError(t, err)
err = multiLimiter.Check(collectionID, internalpb.RateType(rt), math.MaxInt)
assert.ErrorIs(t, err, merr.ErrServiceRateLimit)
}
}
@ -88,19 +89,19 @@ func TestMultiRateLimiter(t *testing.T) {
}
for _, rt := range internalpb.RateType_value {
if IsDDLRequest(internalpb.RateType(rt)) {
errCode := multiLimiter.Check(1, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(1, internalpb.RateType(rt), 5)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(1, internalpb.RateType(rt), 5)
assert.Equal(t, commonpb.ErrorCode_RateLimit, errCode)
err := multiLimiter.Check(1, internalpb.RateType(rt), 1)
assert.NoError(t, err)
err = multiLimiter.Check(1, internalpb.RateType(rt), 5)
assert.NoError(t, err)
err = multiLimiter.Check(1, internalpb.RateType(rt), 5)
assert.ErrorIs(t, err, merr.ErrServiceRateLimit)
} else {
errCode := multiLimiter.Check(1, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(2, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
errCode = multiLimiter.Check(3, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_RateLimit, errCode)
err := multiLimiter.Check(1, internalpb.RateType(rt), 1)
assert.NoError(t, err)
err = multiLimiter.Check(2, internalpb.RateType(rt), 1)
assert.NoError(t, err)
err = multiLimiter.Check(3, internalpb.RateType(rt), 1)
assert.ErrorIs(t, err, merr.ErrServiceRateLimit)
}
}
Params.Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, bak)
@ -112,8 +113,8 @@ func TestMultiRateLimiter(t *testing.T) {
bak := Params.QuotaConfig.QuotaAndLimitsEnabled.GetValue()
paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "false")
for _, rt := range internalpb.RateType_value {
errCode := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
err := multiLimiter.Check(collectionID, internalpb.RateType(rt), 1)
assert.NoError(t, err)
}
Params.Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, bak)
})
@ -125,8 +126,8 @@ func TestMultiRateLimiter(t *testing.T) {
multiLimiter := NewMultiRateLimiter()
bak := Params.QuotaConfig.QuotaAndLimitsEnabled.GetValue()
paramtable.Get().Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, "true")
errCode := multiLimiter.Check(collectionID, internalpb.RateType_DMLInsert, 1*1024*1024)
assert.Equal(t, commonpb.ErrorCode_Success, errCode)
err := multiLimiter.Check(collectionID, internalpb.RateType_DMLInsert, 1*1024*1024)
assert.NoError(t, err)
Params.Save(Params.QuotaConfig.QuotaAndLimitsEnabled.Key, bak)
Params.Save(Params.QuotaConfig.DMLMaxInsertRate.Key, bakInsertRate)
}
@ -282,8 +283,8 @@ func TestRateLimiter(t *testing.T) {
},
})
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_ForceDeny, limiter.getErrorCode(internalpb.RateType_DQLQuery))
assert.Equal(t, commonpb.ErrorCode_DiskQuotaExhausted, limiter.getErrorCode(internalpb.RateType_DMLInsert))
assert.ErrorIs(t, limiter.getError(internalpb.RateType_DQLQuery), merr.ErrServiceForceDeny)
assert.Equal(t, limiter.getError(internalpb.RateType_DMLInsert), merr.ErrServiceDiskLimitExceeded)
})
t.Run("tests refresh rate by config", func(t *testing.T) {

View File

@ -21,13 +21,14 @@ import (
"fmt"
"reflect"
"github.com/cockroachdb/errors"
"github.com/golang/protobuf/proto"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/util/merr"
)
// RateLimitInterceptor returns a new unary server interceptors that performs request rate limiting.
@ -38,9 +39,9 @@ func RateLimitInterceptor(limiter types.Limiter) grpc.UnaryServerInterceptor {
return handler(ctx, req)
}
code := limiter.Check(collectionID, rt, n)
if code != commonpb.ErrorCode_Success {
rsp := getFailedResponse(req, rt, code, info.FullMethod)
err = limiter.Check(collectionID, rt, n)
if err != nil {
rsp := getFailedResponse(req, rt, err, info.FullMethod)
if rsp != nil {
return rsp, nil
}
@ -113,31 +114,16 @@ func getRequestInfo(req interface{}) (int64, internalpb.RateType, int, error) {
}
}
// failedStatus returns failed status.
func failedStatus(code commonpb.ErrorCode, reason string) *commonpb.Status {
return &commonpb.Status{
ErrorCode: code,
Reason: reason,
}
}
// failedMutationResult returns failed mutation result.
func failedMutationResult(code commonpb.ErrorCode, reason string) *milvuspb.MutationResult {
func failedMutationResult(err error) *milvuspb.MutationResult {
return &milvuspb.MutationResult{
Status: failedStatus(code, reason),
Status: merr.Status(err),
}
}
// failedBoolResponse returns failed boolean response.
func failedBoolResponse(code commonpb.ErrorCode, reason string) *milvuspb.BoolResponse {
return &milvuspb.BoolResponse{
Status: failedStatus(code, reason),
}
}
func wrapQuotaError(rt internalpb.RateType, errCode commonpb.ErrorCode, fullMethod string) error {
if errCode == commonpb.ErrorCode_RateLimit {
return fmt.Errorf("request is rejected by grpc RateLimiter middleware, please retry later, req: %s", fullMethod)
func wrapQuotaError(rt internalpb.RateType, err error, fullMethod string) error {
if errors.Is(err, merr.ErrServiceRateLimit) {
return errors.Wrapf(err, "request %s is rejected by grpc RateLimiter middleware, please retry later", fullMethod)
}
// deny to write/read
@ -148,40 +134,41 @@ func wrapQuotaError(rt internalpb.RateType, errCode commonpb.ErrorCode, fullMeth
case internalpb.RateType_DQLSearch, internalpb.RateType_DQLQuery:
op = "read"
}
return fmt.Errorf("deny to %s, reason: %s, req: %s", op, GetQuotaErrorString(errCode), fullMethod)
return merr.WrapErrServiceForceDeny(op, err, fullMethod)
}
// getFailedResponse returns failed response.
func getFailedResponse(req interface{}, rt internalpb.RateType, errCode commonpb.ErrorCode, fullMethod string) interface{} {
err := wrapQuotaError(rt, errCode, fullMethod)
func getFailedResponse(req any, rt internalpb.RateType, err error, fullMethod string) any {
err = wrapQuotaError(rt, err, fullMethod)
switch req.(type) {
case *milvuspb.InsertRequest, *milvuspb.DeleteRequest, *milvuspb.UpsertRequest:
return failedMutationResult(errCode, err.Error())
return failedMutationResult(err)
case *milvuspb.ImportRequest:
return &milvuspb.ImportResponse{
Status: failedStatus(errCode, err.Error()),
Status: merr.Status(err),
}
case *milvuspb.SearchRequest:
return &milvuspb.SearchResults{
Status: failedStatus(errCode, err.Error()),
Status: merr.Status(err),
}
case *milvuspb.QueryRequest:
return &milvuspb.QueryResults{
Status: failedStatus(errCode, err.Error()),
Status: merr.Status(err),
}
case *milvuspb.CreateCollectionRequest, *milvuspb.DropCollectionRequest,
*milvuspb.LoadCollectionRequest, *milvuspb.ReleaseCollectionRequest,
*milvuspb.CreatePartitionRequest, *milvuspb.DropPartitionRequest,
*milvuspb.LoadPartitionsRequest, *milvuspb.ReleasePartitionsRequest,
*milvuspb.CreateIndexRequest, *milvuspb.DropIndexRequest:
return failedStatus(errCode, err.Error())
return merr.Status(err)
case *milvuspb.FlushRequest:
return &milvuspb.FlushResponse{
Status: failedStatus(errCode, err.Error()),
Status: merr.Status(err),
}
case *milvuspb.ManualCompactionRequest:
return &milvuspb.ManualCompactionResponse{
Status: failedStatus(errCode, err.Error()),
Status: merr.Status(err),
}
}
return nil

View File

@ -28,6 +28,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/merr"
)
type limiterMock struct {
@ -37,14 +38,14 @@ type limiterMock struct {
quotaStateReasons []commonpb.ErrorCode
}
func (l *limiterMock) Check(collection int64, rt internalpb.RateType, n int) commonpb.ErrorCode {
func (l *limiterMock) Check(collection int64, rt internalpb.RateType, n int) error {
if l.rate == 0 {
return commonpb.ErrorCode_ForceDeny
return merr.ErrServiceForceDeny
}
if l.limit {
return commonpb.ErrorCode_RateLimit
return merr.ErrServiceRateLimit
}
return commonpb.ErrorCode_Success
return nil
}
func TestRateLimitInterceptor(t *testing.T) {
@ -165,24 +166,24 @@ func TestRateLimitInterceptor(t *testing.T) {
})
t.Run("test getFailedResponse", func(t *testing.T) {
testGetFailedResponse := func(req interface{}, rt internalpb.RateType, errCode commonpb.ErrorCode, fullMethod string) {
rsp := getFailedResponse(req, rt, errCode, fullMethod)
testGetFailedResponse := func(req interface{}, rt internalpb.RateType, err error, fullMethod string) {
rsp := getFailedResponse(req, rt, err, fullMethod)
assert.NotNil(t, rsp)
}
testGetFailedResponse(&milvuspb.DeleteRequest{}, internalpb.RateType_DMLDelete, commonpb.ErrorCode_ForceDeny, "delete")
testGetFailedResponse(&milvuspb.UpsertRequest{}, internalpb.RateType_DMLUpsert, commonpb.ErrorCode_ForceDeny, "upsert")
testGetFailedResponse(&milvuspb.ImportRequest{}, internalpb.RateType_DMLBulkLoad, commonpb.ErrorCode_MemoryQuotaExhausted, "import")
testGetFailedResponse(&milvuspb.SearchRequest{}, internalpb.RateType_DQLSearch, commonpb.ErrorCode_DiskQuotaExhausted, "search")
testGetFailedResponse(&milvuspb.QueryRequest{}, internalpb.RateType_DQLQuery, commonpb.ErrorCode_ForceDeny, "query")
testGetFailedResponse(&milvuspb.CreateCollectionRequest{}, internalpb.RateType_DDLCollection, commonpb.ErrorCode_RateLimit, "createCollection")
testGetFailedResponse(&milvuspb.FlushRequest{}, internalpb.RateType_DDLFlush, commonpb.ErrorCode_RateLimit, "flush")
testGetFailedResponse(&milvuspb.ManualCompactionRequest{}, internalpb.RateType_DDLCompaction, commonpb.ErrorCode_RateLimit, "compaction")
testGetFailedResponse(&milvuspb.DeleteRequest{}, internalpb.RateType_DMLDelete, merr.ErrServiceForceDeny, "delete")
testGetFailedResponse(&milvuspb.UpsertRequest{}, internalpb.RateType_DMLUpsert, merr.ErrServiceForceDeny, "upsert")
testGetFailedResponse(&milvuspb.ImportRequest{}, internalpb.RateType_DMLBulkLoad, merr.ErrServiceMemoryLimitExceeded, "import")
testGetFailedResponse(&milvuspb.SearchRequest{}, internalpb.RateType_DQLSearch, merr.ErrServiceDiskLimitExceeded, "search")
testGetFailedResponse(&milvuspb.QueryRequest{}, internalpb.RateType_DQLQuery, merr.ErrServiceForceDeny, "query")
testGetFailedResponse(&milvuspb.CreateCollectionRequest{}, internalpb.RateType_DDLCollection, merr.ErrServiceRateLimit, "createCollection")
testGetFailedResponse(&milvuspb.FlushRequest{}, internalpb.RateType_DDLFlush, merr.ErrServiceRateLimit, "flush")
testGetFailedResponse(&milvuspb.ManualCompactionRequest{}, internalpb.RateType_DDLCompaction, merr.ErrServiceRateLimit, "compaction")
// test illegal
rsp := getFailedResponse(&milvuspb.SearchResults{}, internalpb.RateType_DQLSearch, commonpb.ErrorCode_UnexpectedError, "method")
rsp := getFailedResponse(&milvuspb.SearchResults{}, internalpb.RateType_DQLSearch, merr.OldCodeToMerr(commonpb.ErrorCode_UnexpectedError), "method")
assert.Nil(t, rsp)
rsp = getFailedResponse(nil, internalpb.RateType_DQLSearch, commonpb.ErrorCode_UnexpectedError, "method")
rsp = getFailedResponse(nil, internalpb.RateType_DQLSearch, merr.OldCodeToMerr(commonpb.ErrorCode_UnexpectedError), "method")
assert.Nil(t, rsp)
})

View File

@ -43,7 +43,7 @@ type TimeTickProvider interface {
// If Limit function return true, the request will be rejected.
// Otherwise, the request will pass. Limit also returns limit of limiter.
type Limiter interface {
Check(collectionID int64, rt internalpb.RateType, n int) commonpb.ErrorCode
Check(collectionID int64, rt internalpb.RateType, n int) error
}
// Component is the interface all services implement

View File

@ -40,6 +40,8 @@ var (
ErrServiceInternal = newMilvusError("service internal error", 5, false) // Never return this error out of Milvus
ErrCrossClusterRouting = newMilvusError("cross cluster routing", 6, false)
ErrServiceDiskLimitExceeded = newMilvusError("disk limit exceeded", 7, false)
ErrServiceRateLimit = newMilvusError("rate limit exceeded", 8, true)
ErrServiceForceDeny = newMilvusError("force deny", 9, false)
// Collection related
ErrCollectionNotFound = newMilvusError("collection not found", 100, false)

View File

@ -135,6 +135,19 @@ func (s *ErrSuite) TestWrap() {
s.ErrorIs(WrapErrFieldNotFound("meta", "failed to get field"), ErrFieldNotFound)
}
func (s *ErrSuite) TestOldCode() {
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_NotReadyServe), ErrServiceNotReady)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_CollectionNotExists), ErrCollectionNotFound)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_IllegalArgument), ErrParameterInvalid)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_NodeIDNotMatch), ErrNodeNotMatch)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_InsufficientMemoryToLoad), ErrServiceMemoryLimitExceeded)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_MemoryQuotaExhausted), ErrServiceMemoryLimitExceeded)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_DiskQuotaExhausted), ErrServiceDiskLimitExceeded)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_RateLimit), ErrServiceRateLimit)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_ForceDeny), ErrServiceForceDeny)
s.ErrorIs(OldCodeToMerr(commonpb.ErrorCode_UnexpectedError), errUnexpected)
}
func (s *ErrSuite) TestCombine() {
var (
errFirst = errors.New("first")

View File

@ -101,23 +101,67 @@ func oldCode(code int32) commonpb.ErrorCode {
switch code {
case ErrServiceNotReady.code():
return commonpb.ErrorCode_NotReadyServe
case ErrCollectionNotFound.code():
return commonpb.ErrorCode_CollectionNotExists
case ErrParameterInvalid.code():
return commonpb.ErrorCode_IllegalArgument
case ErrNodeNotMatch.code():
return commonpb.ErrorCode_NodeIDNotMatch
case ErrCollectionNotFound.code(), ErrPartitionNotFound.code(), ErrReplicaNotFound.code():
return commonpb.ErrorCode_MetaFailed
case ErrReplicaNotAvailable.code(), ErrChannelNotAvailable.code(), ErrNodeNotAvailable.code():
return commonpb.ErrorCode_NoReplicaAvailable
case ErrServiceMemoryLimitExceeded.code():
return commonpb.ErrorCode_InsufficientMemoryToLoad
case ErrServiceRateLimit.code():
return commonpb.ErrorCode_RateLimit
case ErrServiceForceDeny.code():
return commonpb.ErrorCode_ForceDeny
default:
return commonpb.ErrorCode_UnexpectedError
}
}
func OldCodeToMerr(code commonpb.ErrorCode) error {
switch code {
case commonpb.ErrorCode_NotReadyServe:
return ErrServiceNotReady
case commonpb.ErrorCode_CollectionNotExists:
return ErrCollectionNotFound
case commonpb.ErrorCode_IllegalArgument:
return ErrParameterInvalid
case commonpb.ErrorCode_NodeIDNotMatch:
return ErrNodeNotMatch
case commonpb.ErrorCode_InsufficientMemoryToLoad, commonpb.ErrorCode_MemoryQuotaExhausted:
return ErrServiceMemoryLimitExceeded
case commonpb.ErrorCode_DiskQuotaExhausted:
return ErrServiceDiskLimitExceeded
case commonpb.ErrorCode_RateLimit:
return ErrServiceRateLimit
case commonpb.ErrorCode_ForceDeny:
return ErrServiceForceDeny
default:
return errUnexpected
}
}
func Ok(status *commonpb.Status) bool {
return status.GetErrorCode() == commonpb.ErrorCode_Success && status.GetCode() == 0
}
@ -213,6 +257,17 @@ func WrapErrServiceDiskLimitExceeded(predict, limit float32, msg ...string) erro
return err
}
func WrapErrServiceRateLimit(rate float64) error {
err := errors.Wrapf(ErrServiceRateLimit, "rate=%v", rate)
return err
}
func WrapErrServiceForceDeny(op string, reason error, method string) error {
err := errors.Wrapf(ErrServiceForceDeny, "deny to %s, reason: %s, req: %s", op, reason.Error(), method)
return err
}
// database related
func WrapErrDatabaseNotFound(database any, msg ...string) error {
err := wrapWithField(ErrDatabaseNotFound, "database", database)
if len(msg) > 0 {