You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by ha...@apache.org on 2023/02/02 11:58:12 UTC

[skywalking-banyandb] branch main updated: Fix metadata-relevant bugs (#246)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new e98f6c6c Fix metadata-relevant bugs  (#246)
e98f6c6c is described below

commit e98f6c6c25792692560c2e3b28dfdedd3a60327f
Author: HHoflittlefish777 <77...@users.noreply.github.com>
AuthorDate: Thu Feb 2 19:58:06 2023 +0800

    Fix metadata-relevant bugs  (#246)
    
    * fix metadata bug
    * add document
---
 api/proto/banyandb/database/v1/schema.proto     | 59 +++++++++++++------------
 api/proto/banyandb/property/v1/property.proto   |  9 ++--
 banyand/metadata/schema/group.go                |  4 ++
 banyand/metadata/schema/index.go                |  7 +++
 banyand/metadata/schema/measure.go              | 15 +++++++
 banyand/metadata/schema/property.go             | 12 ++++-
 banyand/metadata/schema/stream.go               |  4 ++
 banyand/metadata/schema/topn.go                 |  4 ++
 bydbctl/internal/cmd/index_rule_binding_test.go | 24 ++++++++--
 bydbctl/internal/cmd/index_rule_test.go         | 15 +++++--
 bydbctl/internal/cmd/measure_test.go            | 23 +++++++++-
 bydbctl/internal/cmd/property_test.go           | 13 ++++++
 bydbctl/internal/cmd/stream_test.go             | 18 +++++++-
 docs/api-reference.md                           |  4 +-
 docs/concept/data-model.md                      | 10 +++++
 test/integration/other/property_test.go         |  5 +++
 16 files changed, 180 insertions(+), 46 deletions(-)

diff --git a/api/proto/banyandb/database/v1/schema.proto b/api/proto/banyandb/database/v1/schema.proto
index 929c77b5..604c7a0b 100644
--- a/api/proto/banyandb/database/v1/schema.proto
+++ b/api/proto/banyandb/database/v1/schema.proto
@@ -22,6 +22,7 @@ package banyandb.database.v1;
 import "banyandb/common/v1/common.proto";
 import "banyandb/model/v1/query.proto";
 import "google/protobuf/timestamp.proto";
+import "validate/validate.proto";
 
 option go_package = "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1";
 option java_package = "org.apache.skywalking.banyandb.database.v1";
@@ -37,14 +38,14 @@ enum TagType {
 }
 
 message TagFamilySpec {
-  string name = 1;
+  string name = 1 [(validate.rules).string.min_len = 1];
   // tags defines accepted tags
-  repeated TagSpec tags = 2;
+  repeated TagSpec tags = 2 [(validate.rules).repeated.min_items = 1];
 }
 
 message TagSpec {
-  string name = 1;
-  TagType type = 2;
+  string name = 1 [(validate.rules).string.min_len = 1];
+  TagType type = 2 [(validate.rules).enum.defined_only = true];
   // indexed_only indicates whether the tag is stored
   // True: It's indexed only, but not stored
   // False: it's stored and indexed
@@ -54,17 +55,17 @@ message TagSpec {
 // Stream intends to store streaming data, for example, traces or logs
 message Stream {
   // metadata is the identity of a trace series
-  common.v1.Metadata metadata = 1;
+  common.v1.Metadata metadata = 1 [(validate.rules).message.required = true];
   // tag_families
-  repeated TagFamilySpec tag_families = 2;
+  repeated TagFamilySpec tag_families = 2 [(validate.rules).repeated.min_items = 1];
   // entity indicates how to generate a series and shard a stream
-  Entity entity = 3;
+  Entity entity = 3 [(validate.rules).message.required = true];
   // updated_at indicates when the stream is updated
   google.protobuf.Timestamp updated_at = 4;
 }
 
 message Entity {
-  repeated string tag_names = 1;
+  repeated string tag_names = 1 [(validate.rules).repeated.min_items = 1];
 }
 
 enum FieldType {
@@ -88,25 +89,25 @@ enum CompressionMethod {
 // FieldSpec is the specification of field
 message FieldSpec {
   // name is the identity of a field
-  string name = 1;
+  string name = 1 [(validate.rules).string.min_len = 1];
   // field_type denotes the type of field value
-  FieldType field_type = 2;
+  FieldType field_type = 2 [(validate.rules).enum.defined_only = true];
   // encoding_method indicates how to encode data during writing
-  EncodingMethod encoding_method = 3;
+  EncodingMethod encoding_method = 3 [(validate.rules).enum.defined_only = true];
   // compression_method indicates how to compress data during writing
-  CompressionMethod compression_method = 4;
+  CompressionMethod compression_method = 4 [(validate.rules).enum.defined_only = true];
 }
 
 // Measure intends to store data point
 message Measure {
   // metadata is the identity of a measure
-  common.v1.Metadata metadata = 1;
+  common.v1.Metadata metadata = 1 [(validate.rules).message.required = true];
   // tag_families are for filter measures
-  repeated TagFamilySpec tag_families = 2;
+  repeated TagFamilySpec tag_families = 2 [(validate.rules).repeated.min_items = 1];
   // fields denote measure values
   repeated FieldSpec fields = 3;
   // entity indicates which tags will be to generate a series and shard a measure
-  Entity entity = 4;
+  Entity entity = 4 [(validate.rules).message.required = true];
   // interval indicates how frequently to send a data point
   // valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h", "d".
   string interval = 5;
@@ -117,15 +118,16 @@ message Measure {
 // TopNAggregation generates offline TopN statistics for a measure's TopN approximation
 message TopNAggregation {
   // metadata is the identity of an aggregation
-  common.v1.Metadata metadata = 1;
+  common.v1.Metadata metadata = 1 [(validate.rules).message.required = true];
   // source_measure denotes the data source of this aggregation
-  common.v1.Metadata source_measure = 2;
+  common.v1.Metadata source_measure = 2 [(validate.rules).message.required = true];
   // field_name is the name of field used for ranking
-  string field_name = 3;
+  string field_name = 3 [(validate.rules).string.min_len = 1];
   // field_value_sort indicates how to sort fields
   // ASC: bottomN
   // DESC: topN
   // UNSPECIFIED: topN + bottomN
+  // todo validate plugin exist bug https://github.com/bufbuild/protoc-gen-validate/issues/672
   model.v1.Sort field_value_sort = 4;
   // group_by_tag_names groups data points into statistical counters
   repeated string group_by_tag_names = 5;
@@ -143,11 +145,11 @@ message TopNAggregation {
 // IndexRule should bind to a subject through an IndexRuleBinding to generate proper indices.
 message IndexRule {
   // metadata define the rule's identity
-  common.v1.Metadata metadata = 1;
+  common.v1.Metadata metadata = 1 [(validate.rules).message.required = true];
   // tags are the combination that refers to an indexed object
   // If the elements in tags are more than 1, the object will generate a multi-tag index
   // Caveat: All tags in a multi-tag MUST have an identical IndexType
-  repeated string tags = 2;
+  repeated string tags = 2 [(validate.rules).repeated.min_items = 1];
   // Type determine the index structure under the hood
   enum Type {
     TYPE_UNSPECIFIED = 0;
@@ -155,14 +157,14 @@ message IndexRule {
     TYPE_INVERTED = 2;
   }
   // type is the IndexType of this IndexObject.
-  Type type = 3;
+  Type type = 3 [(validate.rules).enum.defined_only = true];
   enum Location {
     LOCATION_UNSPECIFIED = 0;
     LOCATION_SERIES = 1;
     LOCATION_GLOBAL = 2;
   }
   // location indicates where to store index.
-  Location location = 4;
+  Location location = 4 [(validate.rules).enum.defined_only = true];
   // updated_at indicates when the IndexRule is updated
   google.protobuf.Timestamp updated_at = 5;
   enum Analyzer {
@@ -183,9 +185,10 @@ message IndexRule {
 // Subject defines which stream or measure would generate indices
 message Subject {
   // catalog is where the subject belongs to
+  // todo validate plugin exist bug https://github.com/bufbuild/protoc-gen-validate/issues/672
   common.v1.Catalog catalog = 1;
   // name refers to a stream or measure in a particular catalog
-  string name = 2;
+  string name = 2 [(validate.rules).string.min_len = 1];
 }
 
 // IndexRuleBinding is a bridge to connect severalIndexRules to a subject
@@ -193,16 +196,16 @@ message Subject {
 // to control how to generate time series indices.
 message IndexRuleBinding {
   // metadata is the identity of this binding
-  common.v1.Metadata metadata = 1;
+  common.v1.Metadata metadata = 1 [(validate.rules).message.required = true];
   // rules refers to the IndexRule
-  repeated string rules = 2;
+  repeated string rules = 2 [(validate.rules).repeated.min_items = 1];
   // subject indicates the subject of binding action
-  Subject subject = 3;
+  Subject subject = 3 [(validate.rules).message.required = true];
   // begin_at_nanoseconds is the timestamp, after which the binding will be active
-  google.protobuf.Timestamp begin_at = 4;
+  google.protobuf.Timestamp begin_at = 4 [(validate.rules).timestamp.required = true];
   // expire_at_nanoseconds it the timestamp, after which the binding will be inactive
   // expire_at_nanoseconds must be larger than begin_at_nanoseconds
-  google.protobuf.Timestamp expire_at = 5;
+  google.protobuf.Timestamp expire_at = 5 [(validate.rules).timestamp.required = true];
   // updated_at indicates when the IndexRuleBinding is updated
   google.protobuf.Timestamp updated_at = 6;
 }
diff --git a/api/proto/banyandb/property/v1/property.proto b/api/proto/banyandb/property/v1/property.proto
index 142ecb81..6e6b5d86 100644
--- a/api/proto/banyandb/property/v1/property.proto
+++ b/api/proto/banyandb/property/v1/property.proto
@@ -22,6 +22,7 @@ package banyandb.property.v1;
 import "banyandb/common/v1/common.proto";
 import "banyandb/model/v1/query.proto";
 import "google/protobuf/timestamp.proto";
+import "validate/validate.proto";
 
 option go_package = "github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1";
 option java_package = "org.apache.skywalking.banyandb.property.v1";
@@ -29,17 +30,17 @@ option java_package = "org.apache.skywalking.banyandb.property.v1";
 // Metadata is for multi-tenant use
 message Metadata {
   // container is created when it receives the first property
-  common.v1.Metadata container = 1;
+  common.v1.Metadata container = 1 [(validate.rules).message.required = true];
   // id identifies a property
-  string id = 2;
+  string id = 2 [(validate.rules).string.min_len = 1];
 }
 
 // Property stores the user defined data
 message Property {
   // metadata is the identity of a property
-  Metadata metadata = 1;
+  Metadata metadata = 1 [(validate.rules).message.required = true];
   // tag stores the content of a property
-  repeated model.v1.Tag tags = 2;
+  repeated model.v1.Tag tags = 2 [(validate.rules).repeated.min_items = 1];
   // updated_at indicates when the property is updated
   google.protobuf.Timestamp updated_at = 3;
 }
diff --git a/banyand/metadata/schema/group.go b/banyand/metadata/schema/group.go
index e6ed49f7..584b75a0 100644
--- a/banyand/metadata/schema/group.go
+++ b/banyand/metadata/schema/group.go
@@ -24,6 +24,7 @@ import (
 	"github.com/pkg/errors"
 	clientv3 "go.etcd.io/etcd/client/v3"
 	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/types/known/timestamppb"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 )
@@ -87,6 +88,9 @@ func (e *etcdSchemaRegistry) DeleteGroup(ctx context.Context, group string) (boo
 }
 
 func (e *etcdSchemaRegistry) CreateGroup(ctx context.Context, group *commonv1.Group) error {
+	if group.UpdatedAt != nil {
+		group.UpdatedAt = timestamppb.Now()
+	}
 	return e.create(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind: KindGroup,
diff --git a/banyand/metadata/schema/index.go b/banyand/metadata/schema/index.go
index d00166e4..f8e38809 100644
--- a/banyand/metadata/schema/index.go
+++ b/banyand/metadata/schema/index.go
@@ -22,6 +22,7 @@ import (
 	"hash/crc32"
 
 	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/types/known/timestamppb"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
@@ -58,6 +59,9 @@ func (e *etcdSchemaRegistry) ListIndexRuleBinding(ctx context.Context, opt ListO
 }
 
 func (e *etcdSchemaRegistry) CreateIndexRuleBinding(ctx context.Context, indexRuleBinding *databasev1.IndexRuleBinding) error {
+	if indexRuleBinding.UpdatedAt != nil {
+		indexRuleBinding.UpdatedAt = timestamppb.Now()
+	}
 	return e.create(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindIndexRuleBinding,
@@ -122,6 +126,9 @@ func (e *etcdSchemaRegistry) ListIndexRule(ctx context.Context, opt ListOpt) ([]
 }
 
 func (e *etcdSchemaRegistry) CreateIndexRule(ctx context.Context, indexRule *databasev1.IndexRule) error {
+	if indexRule.UpdatedAt != nil {
+		indexRule.UpdatedAt = timestamppb.Now()
+	}
 	if indexRule.Metadata.Id == 0 {
 		buf := []byte(indexRule.Metadata.Group)
 		buf = append(buf, indexRule.Metadata.Name...)
diff --git a/banyand/metadata/schema/measure.go b/banyand/metadata/schema/measure.go
index b4276b71..d545b11d 100644
--- a/banyand/metadata/schema/measure.go
+++ b/banyand/metadata/schema/measure.go
@@ -21,11 +21,13 @@ import (
 	"context"
 	"time"
 
+	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
 	"google.golang.org/protobuf/types/known/timestamppb"
 
 	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/timestamp"
 )
 
 const (
@@ -59,6 +61,14 @@ func (e *etcdSchemaRegistry) ListMeasure(ctx context.Context, opt ListOpt) ([]*d
 }
 
 func (e *etcdSchemaRegistry) CreateMeasure(ctx context.Context, measure *databasev1.Measure) error {
+	if measure.UpdatedAt != nil {
+		measure.UpdatedAt = timestamppb.Now()
+	}
+	if measure.GetInterval() != "" {
+		if _, err := timestamp.ParseDuration(measure.GetInterval()); err != nil {
+			return errors.Wrap(err, "interval is malformed")
+		}
+	}
 	if err := e.create(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindMeasure,
@@ -126,6 +136,11 @@ func (e *etcdSchemaRegistry) CreateMeasure(ctx context.Context, measure *databas
 }
 
 func (e *etcdSchemaRegistry) UpdateMeasure(ctx context.Context, measure *databasev1.Measure) error {
+	if measure.GetInterval() != "" {
+		if _, err := timestamp.ParseDuration(measure.GetInterval()); err != nil {
+			return errors.Wrap(err, "interval is malformed")
+		}
+	}
 	if err := e.update(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindMeasure,
diff --git a/banyand/metadata/schema/property.go b/banyand/metadata/schema/property.go
index b4e8024d..0333af5e 100644
--- a/banyand/metadata/schema/property.go
+++ b/banyand/metadata/schema/property.go
@@ -19,9 +19,10 @@ package schema
 
 import (
 	"context"
-	"errors"
 
+	"github.com/pkg/errors"
 	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/types/known/timestamppb"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	propertyv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1"
@@ -86,10 +87,17 @@ func (e *etcdSchemaRegistry) ListProperty(ctx context.Context, container *common
 
 func (e *etcdSchemaRegistry) ApplyProperty(ctx context.Context, property *propertyv1.Property, strategy propertyv1.ApplyRequest_Strategy) (bool, uint32, error) {
 	m := transformKey(property.GetMetadata())
+	group := m.GetGroup()
+	if _, getGroupErr := e.GetGroup(ctx, group); getGroupErr != nil {
+		return false, 0, errors.Wrap(getGroupErr, "group is not exist")
+	}
+	if property.UpdatedAt != nil {
+		property.UpdatedAt = timestamppb.Now()
+	}
 	md := Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindProperty,
-			Group: m.GetGroup(),
+			Group: group,
 			Name:  m.GetName(),
 		},
 		Spec: property,
diff --git a/banyand/metadata/schema/stream.go b/banyand/metadata/schema/stream.go
index 57427e33..4f8c8d70 100644
--- a/banyand/metadata/schema/stream.go
+++ b/banyand/metadata/schema/stream.go
@@ -21,6 +21,7 @@ import (
 	"context"
 
 	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/types/known/timestamppb"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
@@ -65,6 +66,9 @@ func (e *etcdSchemaRegistry) UpdateStream(ctx context.Context, stream *databasev
 }
 
 func (e *etcdSchemaRegistry) CreateStream(ctx context.Context, stream *databasev1.Stream) error {
+	if stream.UpdatedAt != nil {
+		stream.UpdatedAt = timestamppb.Now()
+	}
 	group := stream.Metadata.GetGroup()
 	_, err := e.GetGroup(ctx, group)
 	if err != nil {
diff --git a/banyand/metadata/schema/topn.go b/banyand/metadata/schema/topn.go
index 4ba89b90..f1c341f7 100644
--- a/banyand/metadata/schema/topn.go
+++ b/banyand/metadata/schema/topn.go
@@ -21,6 +21,7 @@ import (
 	"context"
 
 	"google.golang.org/protobuf/proto"
+	"google.golang.org/protobuf/types/known/timestamppb"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
@@ -54,6 +55,9 @@ func (e *etcdSchemaRegistry) ListTopNAggregation(ctx context.Context, opt ListOp
 }
 
 func (e *etcdSchemaRegistry) CreateTopNAggregation(ctx context.Context, topNAggregation *databasev1.TopNAggregation) error {
+	if topNAggregation.UpdatedAt != nil {
+		topNAggregation.UpdatedAt = timestamppb.Now()
+	}
 	return e.create(ctx, Metadata{
 		TypeMeta: TypeMeta{
 			Kind:  KindTopNAggregation,
diff --git a/bydbctl/internal/cmd/index_rule_binding_test.go b/bydbctl/internal/cmd/index_rule_binding_test.go
index 7e929683..1b8e4509 100644
--- a/bydbctl/internal/cmd/index_rule_binding_test.go
+++ b/bydbctl/internal/cmd/index_rule_binding_test.go
@@ -74,9 +74,15 @@ resource_opts:
 metadata:
   name: name1
   group: group1
+rules: [
+    "service_id",
+    "searchable_name"
+  ]
 subject:
   catalog: CATALOG_STREAM
-  name: stream1`))
+  name: stream1
+begin_at: 2021-04-15T01:30:15.01Z
+expire_at: 2121-04-15T01:30:15.01Z`))
 			return capturer.CaptureStdout(func() {
 				err := rootCmd.Execute()
 				if err != nil {
@@ -107,9 +113,15 @@ subject:
 metadata:
   name: name1
   group: group1
+rules: [
+    "service_id",
+    "searchable_name"
+  ]
 subject:
   catalog: CATALOG_STREAM
-  name: stream2`))
+  name: stream2
+begin_at: 2021-04-15T01:30:15.01Z
+expire_at: 2121-04-15T01:30:15.01Z`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
@@ -148,9 +160,15 @@ subject:
 metadata:
   name: name2
   group: group1
+rules: [
+    "service_id",
+    "searchable_name"
+  ]
 subject:
   catalog: CATALOG_STREAM
-  name: stream2`))
+  name: stream2
+begin_at: 2021-04-15T01:30:15.01Z
+expire_at: 2121-04-15T01:30:15.01Z`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
diff --git a/bydbctl/internal/cmd/index_rule_test.go b/bydbctl/internal/cmd/index_rule_test.go
index f159b5ce..166f46fd 100644
--- a/bydbctl/internal/cmd/index_rule_test.go
+++ b/bydbctl/internal/cmd/index_rule_test.go
@@ -73,7 +73,10 @@ resource_opts:
 			rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name1
-  group: group1`))
+  group: group1
+tags: ["layer"]
+type: TYPE_INVERTED
+location: LOCATION_SERIES`))
 			return capturer.CaptureStdout(func() {
 				err := rootCmd.Execute()
 				if err != nil {
@@ -102,7 +105,10 @@ metadata:
 		rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name1
-  group: group1`))
+  group: group1
+tags: ["layer"]
+type: TYPE_INVERTED
+location: LOCATION_SERIES`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
@@ -139,7 +145,10 @@ metadata:
 		rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name2
-  group: group1`))
+  group: group1
+tags: ["layer"]
+type: TYPE_INVERTED
+location: LOCATION_SERIES`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
diff --git a/bydbctl/internal/cmd/measure_test.go b/bydbctl/internal/cmd/measure_test.go
index cfb369b4..fb24334e 100644
--- a/bydbctl/internal/cmd/measure_test.go
+++ b/bydbctl/internal/cmd/measure_test.go
@@ -80,7 +80,14 @@ resource_opts:
 			rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name1
-  group: group1`))
+  group: group1
+tag_families:
+  - name: default
+    tags:
+      - name: id
+        type: TAG_TYPE_ID
+entity:
+  tagNames: ["tag1"]`))
 			return capturer.CaptureStdout(func() {
 				err := rootCmd.Execute()
 				if err != nil {
@@ -109,6 +116,11 @@ metadata:
 metadata:
   name: name1
   group: group1
+tag_families:
+  - name: default
+    tags:
+      - name: id
+        type: TAG_TYPE_ID
 entity:
   tagNames: ["tag1"]`))
 		out := capturer.CaptureStdout(func() {
@@ -148,7 +160,14 @@ entity:
 		rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name2
-  group: group1`))
+  group: group1
+tag_families:
+  - name: default
+    tags:
+      - name: id
+        type: TAG_TYPE_ID
+entity:
+  tagNames: ["tag1"]`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
diff --git a/bydbctl/internal/cmd/property_test.go b/bydbctl/internal/cmd/property_test.go
index 74c4c8d5..614375d1 100644
--- a/bydbctl/internal/cmd/property_test.go
+++ b/bydbctl/internal/cmd/property_test.go
@@ -81,6 +81,19 @@ tags:
 		// extracting the operation of creating property schema
 		rootCmd = &cobra.Command{Use: "root"}
 		cmd.RootCmdFlags(rootCmd)
+		rootCmd.SetArgs([]string{"group", "create", "-a", addr, "-f", "-"})
+		creatGroup := func() string {
+			rootCmd.SetIn(strings.NewReader(`
+metadata:
+  name: ui-template`))
+			return capturer.CaptureStdout(func() {
+				err := rootCmd.Execute()
+				if err != nil {
+					GinkgoWriter.Printf("execution fails:%v", err)
+				}
+			})
+		}
+		Eventually(creatGroup, flags.EventuallyTimeout).Should(ContainSubstring("group ui-template is created"))
 		rootCmd.SetArgs([]string{"property", "apply", "-a", addr, "-f", "-"})
 		rootCmd.SetIn(strings.NewReader(p1YAML))
 		out := capturer.CaptureStdout(func() {
diff --git a/bydbctl/internal/cmd/stream_test.go b/bydbctl/internal/cmd/stream_test.go
index 24322f1c..36f8330f 100644
--- a/bydbctl/internal/cmd/stream_test.go
+++ b/bydbctl/internal/cmd/stream_test.go
@@ -85,7 +85,9 @@ tagFamilies:
   - name: searchable
     tags: 
       - name: trace_id
-        type: TAG_TYPE_STRING`))
+        type: TAG_TYPE_STRING
+entity:
+  tagNames: ["tag1"]`))
 			return capturer.CaptureStdout(func() {
 				err := rootCmd.Execute()
 				if err != nil {
@@ -115,6 +117,11 @@ tagFamilies:
 metadata:
   name: name1
   group: group1
+tagFamilies:
+  - name: searchable
+    tags: 
+      - name: trace_id
+        type: TAG_TYPE_STRING
 entity:
   tagNames: ["tag1"]`))
 		out := capturer.CaptureStdout(func() {
@@ -154,7 +161,14 @@ entity:
 		rootCmd.SetIn(strings.NewReader(`
 metadata:
   name: name2
-  group: group1`))
+  group: group1
+tagFamilies:
+  - name: searchable
+    tags: 
+      - name: trace_id
+        type: TAG_TYPE_STRING
+entity:
+  tagNames: ["tag1"]`))
 		out := capturer.CaptureStdout(func() {
 			err := rootCmd.Execute()
 			Expect(err).NotTo(HaveOccurred())
diff --git a/docs/api-reference.md b/docs/api-reference.md
index e9692694..6d05c457 100644
--- a/docs/api-reference.md
+++ b/docs/api-reference.md
@@ -989,7 +989,7 @@ Subject defines which stream or measure would generate indices
 
 | Field | Type | Label | Description |
 | ----- | ---- | ----- | ----------- |
-| catalog | [banyandb.common.v1.Catalog](#banyandb-common-v1-Catalog) |  | catalog is where the subject belongs to |
+| catalog | [banyandb.common.v1.Catalog](#banyandb-common-v1-Catalog) |  | catalog is where the subject belongs to todo validate plugin exist bug https://github.com/bufbuild/protoc-gen-validate/issues/672 |
 | name | [string](#string) |  | name refers to a stream or measure in a particular catalog |
 
 
@@ -1041,7 +1041,7 @@ TopNAggregation generates offline TopN statistics for a measure&#39;s TopN appro
 | metadata | [banyandb.common.v1.Metadata](#banyandb-common-v1-Metadata) |  | metadata is the identity of an aggregation |
 | source_measure | [banyandb.common.v1.Metadata](#banyandb-common-v1-Metadata) |  | source_measure denotes the data source of this aggregation |
 | field_name | [string](#string) |  | field_name is the name of field used for ranking |
-| field_value_sort | [banyandb.model.v1.Sort](#banyandb-model-v1-Sort) |  | field_value_sort indicates how to sort fields ASC: bottomN DESC: topN UNSPECIFIED: topN &#43; bottomN |
+| field_value_sort | [banyandb.model.v1.Sort](#banyandb-model-v1-Sort) |  | field_value_sort indicates how to sort fields ASC: bottomN DESC: topN UNSPECIFIED: topN &#43; bottomN todo validate plugin exist bug https://github.com/bufbuild/protoc-gen-validate/issues/672 |
 | group_by_tag_names | [string](#string) | repeated | group_by_tag_names groups data points into statistical counters |
 | criteria | [banyandb.model.v1.Criteria](#banyandb-model-v1-Criteria) |  | criteria select partial data points from measure |
 | counters_number | [int32](#int32) |  | counters_number sets the number of counters to be tracked. The default value is 1000 |
diff --git a/docs/concept/data-model.md b/docs/concept/data-model.md
index fc408672..9e2634d7 100644
--- a/docs/concept/data-model.md
+++ b/docs/concept/data-model.md
@@ -169,6 +169,16 @@ Tags in `group_by_tag_names` are used as dimensions. These tags can be searched
 
 `Property` is a standard key-value store. Users could store their metadata or items on a property and get a [sequential consistency](https://en.wikipedia.org/wiki/Consistency_model#Sequential_consistency) guarantee. BanyanDB's motivation for introducing such a particular structure is to support most APM scenarios that need to store critical data, especially for a distributed database cluster.
 
+We should create group before creating a property.
+
+Creating group.
+
+```yaml
+metadata:
+  name: sw
+```
+Creating property.
+
 ```yaml
 metadata:
   container:
diff --git a/test/integration/other/property_test.go b/test/integration/other/property_test.go
index 2edef426..84de1c0e 100644
--- a/test/integration/other/property_test.go
+++ b/test/integration/other/property_test.go
@@ -28,6 +28,7 @@ import (
 	"google.golang.org/grpc/credentials/insecure"
 
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
+	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
 	modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1"
 	propertyv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/property/v1"
 	"github.com/apache/skywalking-banyandb/pkg/grpchelper"
@@ -50,6 +51,8 @@ var _ = Describe("Property application", func() {
 		var err error
 		conn, err = grpchelper.Conn(addr, 10*time.Second, grpc.WithTransportCredentials(insecure.NewCredentials()))
 		Expect(err).NotTo(HaveOccurred())
+		gClient := databasev1.NewGroupRegistryServiceClient(conn)
+		gClient.Create(context.Background(), &databasev1.GroupRegistryServiceCreateRequest{Group: &commonv1.Group{Metadata: &commonv1.Metadata{Name: "g"}}})
 		client = propertyv1.NewPropertyServiceClient(conn)
 		goods = gleak.Goroutines()
 	})
@@ -112,6 +115,8 @@ var _ = Describe("Property application", func() {
 		var err error
 		conn, err = grpchelper.Conn(addr, 10*time.Second, grpc.WithTransportCredentials(insecure.NewCredentials()))
 		Expect(err).NotTo(HaveOccurred())
+		gClient := databasev1.NewGroupRegistryServiceClient(conn)
+		gClient.Create(context.Background(), &databasev1.GroupRegistryServiceCreateRequest{Group: &commonv1.Group{Metadata: &commonv1.Metadata{Name: "g"}}})
 		client = propertyv1.NewPropertyServiceClient(conn)
 		md = &propertyv1.Metadata{
 			Container: &commonv1.Metadata{