You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by lu...@apache.org on 2022/04/09 14:30:43 UTC

[skywalking-banyandb] 01/01: add grpc errors

This is an automated email from the ASF dual-hosted git repository.

lujiajing pushed a commit to branch polish-error
in repository https://gitbox.apache.org/repos/asf/skywalking-banyandb.git

commit 3499fd069241afc685e23cd4ff4b2a76d11b92e8
Author: Megrez Lu <lu...@gmail.com>
AuthorDate: Sat Apr 9 22:30:32 2022 +0800

    add grpc errors
---
 banyand/metadata/schema/etcd.go     |  9 ++++--
 banyand/metadata/schema/group.go    |  8 ++---
 banyand/metadata/schema/index.go    | 16 +++++-----
 banyand/metadata/schema/measure.go  | 16 +++++-----
 banyand/metadata/schema/property.go | 10 +++----
 banyand/metadata/schema/schema.go   | 14 ++++-----
 banyand/metadata/schema/stream.go   | 10 +++----
 banyand/metadata/schema/topn.go     | 10 +++----
 pkg/schema/error.go                 | 59 +++++++++++++++++++++++++++++++++++++
 9 files changed, 106 insertions(+), 46 deletions(-)

diff --git a/banyand/metadata/schema/etcd.go b/banyand/metadata/schema/etcd.go
index 0d1d58e..e8e40bd 100644
--- a/banyand/metadata/schema/etcd.go
+++ b/banyand/metadata/schema/etcd.go
@@ -32,6 +32,7 @@ import (
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 )
 
 var (
@@ -42,7 +43,6 @@ var (
 	_ Group            = (*etcdSchemaRegistry)(nil)
 	_ Property         = (*etcdSchemaRegistry)(nil)
 
-	ErrEntityNotFound             = errors.New("entity is not found")
 	ErrUnexpectedNumberOfEntities = errors.New("unexpected number of entities")
 	ErrConcurrentModification     = errors.New("concurrent modification of entities")
 
@@ -176,7 +176,7 @@ func (e *etcdSchemaRegistry) get(ctx context.Context, key string, message proto.
 		return err
 	}
 	if resp.Count == 0 {
-		return ErrEntityNotFound
+		return schema.ErrGRPCResourceNotFound
 	}
 	if resp.Count > 1 {
 		return ErrUnexpectedNumberOfEntities
@@ -192,7 +192,7 @@ func (e *etcdSchemaRegistry) get(ctx context.Context, key string, message proto.
 	return nil
 }
 
-func (e *etcdSchemaRegistry) update(ctx context.Context, metadata Metadata) error {
+func (e *etcdSchemaRegistry) update(ctx context.Context, metadata Metadata, allowOverwrite bool) error {
 	key, err := metadata.Key()
 	if err != nil {
 		return err
@@ -210,6 +210,9 @@ func (e *etcdSchemaRegistry) update(ctx context.Context, metadata Metadata) erro
 	}
 	replace := getResp.Count > 0
 	if replace {
+		if !allowOverwrite {
+			return schema.ErrGRPCAlreadyExists
+		}
 		existingVal, innerErr := metadata.Unmarshal(getResp.Kvs[0].Value)
 		if innerErr != nil {
 			return innerErr
diff --git a/banyand/metadata/schema/group.go b/banyand/metadata/schema/group.go
index 786dbb4..644a380 100644
--- a/banyand/metadata/schema/group.go
+++ b/banyand/metadata/schema/group.go
@@ -14,7 +14,7 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
@@ -31,8 +31,6 @@ import (
 var (
 	GroupsKeyPrefix  = "/groups/"
 	GroupMetadataKey = "/__meta_group__"
-
-	ErrGroupAbsent = errors.New("group is absent")
 )
 
 func (e *etcdSchemaRegistry) GetGroup(ctx context.Context, group string) (*commonv1.Group, error) {
@@ -88,14 +86,14 @@ func (e *etcdSchemaRegistry) DeleteGroup(ctx context.Context, group string) (boo
 	return true, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateGroup(ctx context.Context, group *commonv1.Group) error {
+func (e *etcdSchemaRegistry) UpdateGroup(ctx context.Context, group *commonv1.Group, allowOverwrite bool) error {
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind: KindGroup,
 			Name: group.GetMetadata().GetName(),
 		},
 		Spec: group,
-	})
+	}, allowOverwrite)
 }
 
 func formatGroupKey(group string) string {
diff --git a/banyand/metadata/schema/index.go b/banyand/metadata/schema/index.go
index 33dd4cb..6b5d948 100644
--- a/banyand/metadata/schema/index.go
+++ b/banyand/metadata/schema/index.go
@@ -14,17 +14,17 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
 	"context"
 
-	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 )
 
 var (
@@ -42,7 +42,7 @@ func (e *etcdSchemaRegistry) GetIndexRuleBinding(ctx context.Context, metadata *
 
 func (e *etcdSchemaRegistry) ListIndexRuleBinding(ctx context.Context, opt ListOpt) ([]*databasev1.IndexRuleBinding, error) {
 	if opt.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list index rule binding")
+		return nil, schema.BadRequest("group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(opt.Group, IndexRuleBindingKeyPrefix), func() proto.Message {
 		return &databasev1.IndexRuleBinding{}
@@ -57,7 +57,7 @@ func (e *etcdSchemaRegistry) ListIndexRuleBinding(ctx context.Context, opt ListO
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateIndexRuleBinding(ctx context.Context, indexRuleBinding *databasev1.IndexRuleBinding) error {
+func (e *etcdSchemaRegistry) UpdateIndexRuleBinding(ctx context.Context, indexRuleBinding *databasev1.IndexRuleBinding, allowOverwrite bool) error {
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindIndexRuleBinding,
@@ -65,7 +65,7 @@ func (e *etcdSchemaRegistry) UpdateIndexRuleBinding(ctx context.Context, indexRu
 			Group: indexRuleBinding.GetMetadata().GetGroup(),
 		},
 		Spec: indexRuleBinding,
-	})
+	}, allowOverwrite)
 }
 
 func (e *etcdSchemaRegistry) DeleteIndexRuleBinding(ctx context.Context, metadata *commonv1.Metadata) (bool, error) {
@@ -88,7 +88,7 @@ func (e *etcdSchemaRegistry) GetIndexRule(ctx context.Context, metadata *commonv
 
 func (e *etcdSchemaRegistry) ListIndexRule(ctx context.Context, opt ListOpt) ([]*databasev1.IndexRule, error) {
 	if opt.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list index rule")
+		return nil, schema.BadRequest("group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(opt.Group, IndexRuleKeyPrefix), func() proto.Message {
 		return &databasev1.IndexRule{}
@@ -103,7 +103,7 @@ func (e *etcdSchemaRegistry) ListIndexRule(ctx context.Context, opt ListOpt) ([]
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateIndexRule(ctx context.Context, indexRule *databasev1.IndexRule) error {
+func (e *etcdSchemaRegistry) UpdateIndexRule(ctx context.Context, indexRule *databasev1.IndexRule, allowOverwrite bool) error {
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindIndexRule,
@@ -111,7 +111,7 @@ func (e *etcdSchemaRegistry) UpdateIndexRule(ctx context.Context, indexRule *dat
 			Group: indexRule.GetMetadata().GetGroup(),
 		},
 		Spec: indexRule,
-	})
+	}, allowOverwrite)
 }
 
 func (e *etcdSchemaRegistry) DeleteIndexRule(ctx context.Context, metadata *commonv1.Metadata) (bool, error) {
diff --git a/banyand/metadata/schema/measure.go b/banyand/metadata/schema/measure.go
index 33e5c42..77e82a1 100644
--- a/banyand/metadata/schema/measure.go
+++ b/banyand/metadata/schema/measure.go
@@ -14,14 +14,14 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
 	"context"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 	"time"
 
-	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 	"google.golang.org/protobuf/types/known/timestamppb"
 
@@ -44,7 +44,7 @@ func (e *etcdSchemaRegistry) GetMeasure(ctx context.Context, metadata *commonv1.
 
 func (e *etcdSchemaRegistry) ListMeasure(ctx context.Context, opt ListOpt) ([]*databasev1.Measure, error) {
 	if opt.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list measure")
+		return nil, schema.BadRequest("group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(opt.Group, MeasureKeyPrefix), func() proto.Message {
 		return &databasev1.Measure{}
@@ -59,7 +59,7 @@ func (e *etcdSchemaRegistry) ListMeasure(ctx context.Context, opt ListOpt) ([]*d
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databasev1.Measure) error {
+func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databasev1.Measure, allowOverwrite bool) error {
 	if err := e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindMeasure,
@@ -67,7 +67,7 @@ func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databas
 			Name:  measure.GetMetadata().GetName(),
 		},
 		Spec: measure,
-	}); err != nil {
+	}, allowOverwrite); err != nil {
 		return err
 	}
 
@@ -77,14 +77,14 @@ func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databas
 		Group: measure.Metadata.Group,
 	}
 	_, err := e.GetIndexRule(ctx, idIndexRuleMetadata)
-	if errors.Is(err, ErrEntityNotFound) {
+	if schema.IsNotFound(err) {
 		if errIndexRule := e.UpdateIndexRule(ctx, &databasev1.IndexRule{
 			Metadata:  idIndexRuleMetadata,
 			Tags:      []string{TagTypeID},
 			Type:      databasev1.IndexRule_TYPE_TREE,
 			Location:  databasev1.IndexRule_LOCATION_SERIES,
 			UpdatedAt: timestamppb.Now(),
-		}); errIndexRule != nil {
+		}, false); errIndexRule != nil {
 			return errIndexRule
 		}
 	} else if err != nil {
@@ -111,7 +111,7 @@ func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databas
 					BeginAt:   timestamppb.Now(),
 					ExpireAt:  timestamppb.New(time.Now().AddDate(100, 0, 0)),
 					UpdatedAt: timestamppb.Now(),
-				}); errIndexRule != nil {
+				}, true); errIndexRule != nil {
 					return errIndexRule
 				}
 			}
diff --git a/banyand/metadata/schema/property.go b/banyand/metadata/schema/property.go
index 28cc8a5..529382a 100644
--- a/banyand/metadata/schema/property.go
+++ b/banyand/metadata/schema/property.go
@@ -14,17 +14,17 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
 	"context"
 
-	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	propertyv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 )
 
 var PropertyKeyPrefix = "/properties/"
@@ -39,7 +39,7 @@ func (e *etcdSchemaRegistry) GetProperty(ctx context.Context, metadata *property
 
 func (e *etcdSchemaRegistry) ListProperty(ctx context.Context, container *commonv1.Metadata) ([]*propertyv1.Property, error) {
 	if container.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list Property")
+		return nil, schema.BadRequest("container.group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(container.Group, PropertyKeyPrefix+container.Name), func() proto.Message {
 		return &propertyv1.Property{}
@@ -54,7 +54,7 @@ func (e *etcdSchemaRegistry) ListProperty(ctx context.Context, container *common
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateProperty(ctx context.Context, property *propertyv1.Property) error {
+func (e *etcdSchemaRegistry) UpdateProperty(ctx context.Context, property *propertyv1.Property, allowOverwrite bool) error {
 	m := transformKey(property.GetMetadata())
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
@@ -63,7 +63,7 @@ func (e *etcdSchemaRegistry) UpdateProperty(ctx context.Context, property *prope
 			Name:  m.GetName(),
 		},
 		Spec: property,
-	})
+	}, allowOverwrite)
 }
 
 func (e *etcdSchemaRegistry) DeleteProperty(ctx context.Context, metadata *propertyv1.Metadata) (bool, error) {
diff --git a/banyand/metadata/schema/schema.go b/banyand/metadata/schema/schema.go
index c0f4060..45a547d 100644
--- a/banyand/metadata/schema/schema.go
+++ b/banyand/metadata/schema/schema.go
@@ -157,7 +157,7 @@ func (m Metadata) Equal(other proto.Message) bool {
 type Stream interface {
 	GetStream(ctx context.Context, metadata *commonv1.Metadata) (*databasev1.Stream, error)
 	ListStream(ctx context.Context, opt ListOpt) ([]*databasev1.Stream, error)
-	UpdateStream(ctx context.Context, stream *databasev1.Stream) error
+	UpdateStream(ctx context.Context, stream *databasev1.Stream, allowOverwrite bool) error
 	DeleteStream(ctx context.Context, metadata *commonv1.Metadata) (bool, error)
 	RegisterHandler(Kind, EventHandler)
 }
@@ -165,21 +165,21 @@ type Stream interface {
 type IndexRule interface {
 	GetIndexRule(ctx context.Context, metadata *commonv1.Metadata) (*databasev1.IndexRule, error)
 	ListIndexRule(ctx context.Context, opt ListOpt) ([]*databasev1.IndexRule, error)
-	UpdateIndexRule(ctx context.Context, indexRule *databasev1.IndexRule) error
+	UpdateIndexRule(ctx context.Context, indexRule *databasev1.IndexRule, allowOverwrite bool) error
 	DeleteIndexRule(ctx context.Context, metadata *commonv1.Metadata) (bool, error)
 }
 
 type IndexRuleBinding interface {
 	GetIndexRuleBinding(ctx context.Context, metadata *commonv1.Metadata) (*databasev1.IndexRuleBinding, error)
 	ListIndexRuleBinding(ctx context.Context, opt ListOpt) ([]*databasev1.IndexRuleBinding, error)
-	UpdateIndexRuleBinding(ctx context.Context, indexRuleBinding *databasev1.IndexRuleBinding) error
+	UpdateIndexRuleBinding(ctx context.Context, indexRuleBinding *databasev1.IndexRuleBinding, allowOverwrite bool) error
 	DeleteIndexRuleBinding(ctx context.Context, metadata *commonv1.Metadata) (bool, error)
 }
 
 type Measure interface {
 	GetMeasure(ctx context.Context, metadata *commonv1.Metadata) (*databasev1.Measure, error)
 	ListMeasure(ctx context.Context, opt ListOpt) ([]*databasev1.Measure, error)
-	UpdateMeasure(ctx context.Context, measure *databasev1.Measure) error
+	UpdateMeasure(ctx context.Context, measure *databasev1.Measure, allowOverwrite bool) error
 	DeleteMeasure(ctx context.Context, metadata *commonv1.Metadata) (bool, error)
 	RegisterHandler(Kind, EventHandler)
 }
@@ -189,19 +189,19 @@ type Group interface {
 	ListGroup(ctx context.Context) ([]*commonv1.Group, error)
 	// DeleteGroup delete all items belonging to the group
 	DeleteGroup(ctx context.Context, group string) (bool, error)
-	UpdateGroup(ctx context.Context, group *commonv1.Group) error
+	UpdateGroup(ctx context.Context, group *commonv1.Group, allowOverwrite bool) error
 }
 
 type TopNAggregation interface {
 	GetTopNAggregation(ctx context.Context, metadata *commonv1.Metadata) (*databasev1.TopNAggregation, error)
 	ListTopNAggregation(ctx context.Context, opt ListOpt) ([]*databasev1.TopNAggregation, error)
-	UpdateTopNAggregation(ctx context.Context, measure *databasev1.TopNAggregation) error
+	UpdateTopNAggregation(ctx context.Context, measure *databasev1.TopNAggregation, allowOverwrite bool) error
 	DeleteTopNAggregation(ctx context.Context, metadata *commonv1.Metadata) (bool, error)
 }
 
 type Property interface {
 	GetProperty(ctx context.Context, metadata *propertyv1.Metadata) (*propertyv1.Property, error)
 	ListProperty(ctx context.Context, container *commonv1.Metadata) ([]*propertyv1.Property, error)
-	UpdateProperty(ctx context.Context, property *propertyv1.Property) error
+	UpdateProperty(ctx context.Context, property *propertyv1.Property, allowOverwrite bool) error
 	DeleteProperty(ctx context.Context, metadata *propertyv1.Metadata) (bool, error)
 }
diff --git a/banyand/metadata/schema/stream.go b/banyand/metadata/schema/stream.go
index bbcad7c..3af7ed7 100644
--- a/banyand/metadata/schema/stream.go
+++ b/banyand/metadata/schema/stream.go
@@ -14,17 +14,17 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
 	"context"
 
-	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 )
 
 var StreamKeyPrefix = "/streams/"
@@ -39,7 +39,7 @@ func (e *etcdSchemaRegistry) GetStream(ctx context.Context, metadata *commonv1.M
 
 func (e *etcdSchemaRegistry) ListStream(ctx context.Context, opt ListOpt) ([]*databasev1.Stream, error) {
 	if opt.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list stream")
+		return nil, schema.BadRequest("group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(opt.Group, StreamKeyPrefix), func() proto.Message {
 		return &databasev1.Stream{}
@@ -54,7 +54,7 @@ func (e *etcdSchemaRegistry) ListStream(ctx context.Context, opt ListOpt) ([]*da
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateStream(ctx context.Context, stream *databasev1.Stream) error {
+func (e *etcdSchemaRegistry) UpdateStream(ctx context.Context, stream *databasev1.Stream, allowOverwrite bool) error {
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindStream,
@@ -62,7 +62,7 @@ func (e *etcdSchemaRegistry) UpdateStream(ctx context.Context, stream *databasev
 			Name:  stream.GetMetadata().GetName(),
 		},
 		Spec: stream,
-	})
+	}, allowOverwrite)
 }
 
 func (e *etcdSchemaRegistry) DeleteStream(ctx context.Context, metadata *commonv1.Metadata) (bool, error) {
diff --git a/banyand/metadata/schema/topn.go b/banyand/metadata/schema/topn.go
index af79e24..c762ac7 100644
--- a/banyand/metadata/schema/topn.go
+++ b/banyand/metadata/schema/topn.go
@@ -14,17 +14,17 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-//
+
 package schema
 
 import (
 	"context"
 
-	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
+	"github.com/apache/skywalking-banyandb/pkg/schema"
 )
 
 var TopNAggregationKeyPrefix = "/topnagg/"
@@ -39,7 +39,7 @@ func (e *etcdSchemaRegistry) GetTopNAggregation(ctx context.Context, metadata *c
 
 func (e *etcdSchemaRegistry) ListTopNAggregation(ctx context.Context, opt ListOpt) ([]*databasev1.TopNAggregation, error) {
 	if opt.Group == "" {
-		return nil, errors.Wrap(ErrGroupAbsent, "list TopNAggregation")
+		return nil, schema.BadRequest("group", "group should not be empty")
 	}
 	messages, err := e.listWithPrefix(ctx, listPrefixesForEntity(opt.Group, TopNAggregationKeyPrefix), func() proto.Message {
 		return &databasev1.TopNAggregation{}
@@ -54,7 +54,7 @@ func (e *etcdSchemaRegistry) ListTopNAggregation(ctx context.Context, opt ListOp
 	return entities, nil
 }
 
-func (e *etcdSchemaRegistry) UpdateTopNAggregation(ctx context.Context, topNAggregation *databasev1.TopNAggregation) error {
+func (e *etcdSchemaRegistry) UpdateTopNAggregation(ctx context.Context, topNAggregation *databasev1.TopNAggregation, allowOverwrite bool) error {
 	return e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindTopNAggregation,
@@ -62,7 +62,7 @@ func (e *etcdSchemaRegistry) UpdateTopNAggregation(ctx context.Context, topNAggr
 			Name:  topNAggregation.GetMetadata().GetName(),
 		},
 		Spec: topNAggregation,
-	})
+	}, allowOverwrite)
 }
 
 func (e *etcdSchemaRegistry) DeleteTopNAggregation(ctx context.Context, metadata *commonv1.Metadata) (bool, error) {
diff --git a/pkg/schema/error.go b/pkg/schema/error.go
new file mode 100644
index 0000000..b920e7b
--- /dev/null
+++ b/pkg/schema/error.go
@@ -0,0 +1,59 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you 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 schema
+
+import (
+	"fmt"
+
+	"github.com/pkg/errors"
+	"google.golang.org/genproto/googleapis/rpc/errdetails"
+	"google.golang.org/grpc/codes"
+	"google.golang.org/grpc/status"
+)
+
+var (
+	statusGRPCInvalidArgument  = status.New(codes.InvalidArgument, "banyandb: input is invalid")
+	ErrGRPCInvalidArgument     = statusGRPCInvalidArgument.Err()
+	statusGRPCResourceNotFound = status.New(codes.NotFound, "banyandb: resource not found")
+	ErrGRPCResourceNotFound    = statusGRPCResourceNotFound.Err()
+	statusGRPCAlreadyExists    = status.New(codes.AlreadyExists, "banyandb: resource already exists")
+	ErrGRPCAlreadyExists       = statusGRPCAlreadyExists.Err()
+)
+
+func IsNotFound(err error) bool {
+	return errors.Is(err, ErrGRPCResourceNotFound)
+}
+
+// BadRequest creates a gRPC error with error details with type BadRequest,
+// which describes violations in a client request.
+func BadRequest(field, desc string) error {
+	v := &errdetails.BadRequest_FieldViolation{
+		Field:       field,
+		Description: desc,
+	}
+	br := &errdetails.BadRequest{}
+	br.FieldViolations = append(br.FieldViolations, v)
+	st, err := statusGRPCInvalidArgument.WithDetails(br)
+	if err != nil {
+		// If this errored, it will always error
+		// here, so better panic, so we can figure
+		// out why than have this silently passing.
+		panic(fmt.Sprintf("Unexpected error attaching metadata: %v", err))
+	}
+	return st.Err()
+}