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

[skywalking-banyandb-java-client] branch main updated: Add property API (#10)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 4ff23d9  Add property API (#10)
4ff23d9 is described below

commit 4ff23d9c16f53cea9421811b9c4e3f435441ddc2
Author: Jiajing LU <lu...@gmail.com>
AuthorDate: Sat Apr 16 18:53:11 2022 +0800

    Add property API (#10)
---
 README.md                                          |  32 +++++
 .../banyandb/v1/client/BanyanDBClient.java         |  48 +++++++
 .../banyandb/v1/client/PairQueryCondition.java     |  64 ++-------
 .../skywalking/banyandb/v1/client/TagAndValue.java |  97 +++++++++++++-
 .../banyandb/v1/client/metadata/Property.java      |  90 +++++++++++++
 .../banyandb/v1/client/metadata/PropertyStore.java |  94 +++++++++++++
 src/main/proto/banyandb/v1/banyandb-property.proto |  90 +++++++++++++
 .../banyandb/v1/client/BanyanDBClientTestCI.java   |   4 +-
 ...Tests.java => ITBanyanDBMeasureQueryTests.java} |   2 +-
 .../v1/client/ITBanyanDBPropertyTests.java         | 107 +++++++++++++++
 ...nTests.java => ITBanyanDBStreamQueryTests.java} |   2 +-
 .../v1/client/metadata/PropertyStoreTest.java      | 148 +++++++++++++++++++++
 12 files changed, 722 insertions(+), 56 deletions(-)

diff --git a/README.md b/README.md
index 62fd948..accf6e2 100644
--- a/README.md
+++ b/README.md
@@ -225,6 +225,38 @@ MeasureWrite measureWrite = new MeasureWrite("sw_metric", "service_cpm_minute",
 measureBulkWriteProcessor.add(measureWrite);
 ```
 
+## Property APIs
+
+Property APIs are used to store key-value pairs.
+
+### Create/Update
+
+`save` will always succeed whenever the property exists or not.
+The old value will be overwritten if already existed, otherwise a new value will be set.
+
+```java
+Property property = Property.create("default", "sw", "ui_template")
+    .addTag(TagAndValue.newStringTag("name", "hello"))
+    .build();
+this.client.save(property);
+```
+
+### Query
+
+Property can be queried via `Client.findProperty`,
+
+```java
+Property gotProperty = this.client.findProperty("default", "sw", "ui_template");
+```
+
+### Delete
+
+Property can be deleted by calling `Client.deleteProperty`,
+
+```java
+this.client.deleteProperty("default", "sw", "ui_template");
+```
+
 # Compiling project
 > ./mvnw clean package
 
diff --git a/src/main/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClient.java b/src/main/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClient.java
index 70d9a68..98501c9 100644
--- a/src/main/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClient.java
+++ b/src/main/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClient.java
@@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
+import io.grpc.Status;
 import io.grpc.stub.StreamObserver;
 import lombok.AccessLevel;
 import lombok.Getter;
@@ -54,6 +55,8 @@ import org.apache.skywalking.banyandb.v1.client.metadata.IndexRuleMetadataRegist
 import org.apache.skywalking.banyandb.v1.client.metadata.Measure;
 import org.apache.skywalking.banyandb.v1.client.metadata.MeasureMetadataRegistry;
 import org.apache.skywalking.banyandb.v1.client.metadata.MetadataCache;
+import org.apache.skywalking.banyandb.v1.client.metadata.Property;
+import org.apache.skywalking.banyandb.v1.client.metadata.PropertyStore;
 import org.apache.skywalking.banyandb.v1.client.metadata.Stream;
 import org.apache.skywalking.banyandb.v1.client.metadata.StreamMetadataRegistry;
 
@@ -314,6 +317,51 @@ public class BanyanDBClient implements Closeable {
         MetadataCache.INSTANCE.register(measure);
     }
 
+    /**
+     * Create or update the property
+     *
+     * @param property the property to be stored in the BanyanBD
+     */
+    public void save(Property property) throws BanyanDBException {
+        PropertyStore store = new PropertyStore(checkNotNull(this.channel));
+        try {
+            store.get(property.group(), property.name(), property.id());
+            store.update(property);
+        } catch (BanyanDBException ex) {
+            if (ex.getStatus().equals(Status.Code.NOT_FOUND)) {
+                store.create(property);
+                return;
+            }
+            throw ex;
+        }
+    }
+
+    /**
+     * Find property
+     *
+     * @param group group of the metadata
+     * @param name  name of the metadata
+     * @param id    identity of the property
+     * @return property if it can be found
+     */
+    public Property findProperty(String group, String name, String id) throws BanyanDBException {
+        PropertyStore store = new PropertyStore(checkNotNull(this.channel));
+        return store.get(group, name, id);
+    }
+
+    /**
+     * Delete property
+     *
+     * @param group group of the metadata
+     * @param name  name of the metadata
+     * @param id    identity of the property
+     * @return if this property has been deleted
+     */
+    public boolean deleteProperty(String group, String name, String id) throws BanyanDBException {
+        PropertyStore store = new PropertyStore(checkNotNull(this.channel));
+        return store.delete(group, name, id);
+    }
+
     /**
      * Bind index rule to the stream
      *
diff --git a/src/main/java/org/apache/skywalking/banyandb/v1/client/PairQueryCondition.java b/src/main/java/org/apache/skywalking/banyandb/v1/client/PairQueryCondition.java
index ff7540f..f2f2b0c 100644
--- a/src/main/java/org/apache/skywalking/banyandb/v1/client/PairQueryCondition.java
+++ b/src/main/java/org/apache/skywalking/banyandb/v1/client/PairQueryCondition.java
@@ -25,43 +25,32 @@ import java.util.List;
 /**
  * PairQuery represents a query condition, including tag name, operator, and value(s);
  */
-public abstract class PairQueryCondition<T> extends TagAndValue<T> {
+public abstract class PairQueryCondition<T> {
     protected final BanyandbModel.Condition.BinaryOp op;
+    private final TagAndValue<T> tagAndValue;
 
-    private PairQueryCondition(String tagName, BanyandbModel.Condition.BinaryOp op, T value) {
-        super(tagName, value);
+    private PairQueryCondition(BanyandbModel.Condition.BinaryOp op, TagAndValue<T> tagAndValue) {
         this.op = op;
+        this.tagAndValue = tagAndValue;
     }
 
     BanyandbModel.Condition build() {
         return BanyandbModel.Condition.newBuilder()
-                .setName(this.tagName)
+                .setName(this.tagAndValue.getTagName())
                 .setOp(this.op)
-                .setValue(buildTypedPair()).build();
+                .setValue(this.tagAndValue.buildTypedTagValue()).build();
     }
 
-    /**
-     * The various implementations should build different TypedPair
-     *
-     * @return typedPair to be included
-     */
-    protected abstract BanyandbModel.TagValue buildTypedPair();
+    public String getTagName() {
+        return this.tagAndValue.getTagName();
+    }
 
     /**
      * LongQueryCondition represents `tag(Long) $op value` condition.
      */
     public static class LongQueryCondition extends PairQueryCondition<Long> {
         private LongQueryCondition(String tagName, BanyandbModel.Condition.BinaryOp op, Long value) {
-            super(tagName, op, value);
-        }
-
-        @Override
-        protected BanyandbModel.TagValue buildTypedPair() {
-            return BanyandbModel.TagValue.newBuilder()
-                    .setInt(BanyandbModel.Int
-                            .newBuilder()
-                            .setValue(value).build())
-                    .build();
+            super(op, new TagAndValue.LongTagPair(tagName, value));
         }
 
         /**
@@ -142,16 +131,7 @@ public abstract class PairQueryCondition<T> extends TagAndValue<T> {
      */
     public static class StringQueryCondition extends PairQueryCondition<String> {
         private StringQueryCondition(String tagName, BanyandbModel.Condition.BinaryOp op, String value) {
-            super(tagName, op, value);
-        }
-
-        @Override
-        protected BanyandbModel.TagValue buildTypedPair() {
-            return BanyandbModel.TagValue.newBuilder()
-                    .setStr(BanyandbModel.Str
-                            .newBuilder()
-                            .setValue(value).build())
-                    .build();
+            super(op, new TagAndValue.StringTagPair(tagName, value));
         }
 
         /**
@@ -184,16 +164,7 @@ public abstract class PairQueryCondition<T> extends TagAndValue<T> {
      */
     public static class StringArrayQueryCondition extends PairQueryCondition<List<String>> {
         private StringArrayQueryCondition(String tagName, BanyandbModel.Condition.BinaryOp op, List<String> value) {
-            super(tagName, op, value);
-        }
-
-        @Override
-        protected BanyandbModel.TagValue buildTypedPair() {
-            return BanyandbModel.TagValue.newBuilder()
-                    .setStrArray(BanyandbModel.StrArray
-                            .newBuilder()
-                            .addAllValue(value).build())
-                    .build();
+            super(op, new TagAndValue.StringArrayTagPair(tagName, value));
         }
 
         /**
@@ -250,16 +221,7 @@ public abstract class PairQueryCondition<T> extends TagAndValue<T> {
      */
     public static class LongArrayQueryCondition extends PairQueryCondition<List<Long>> {
         private LongArrayQueryCondition(String tagName, BanyandbModel.Condition.BinaryOp op, List<Long> value) {
-            super(tagName, op, value);
-        }
-
-        @Override
-        protected BanyandbModel.TagValue buildTypedPair() {
-            return BanyandbModel.TagValue.newBuilder()
-                    .setIntArray(BanyandbModel.IntArray
-                            .newBuilder()
-                            .addAllValue(value).build())
-                    .build();
+            super(op, new TagAndValue.LongArrayTagPair(tagName, value));
         }
 
         /**
diff --git a/src/main/java/org/apache/skywalking/banyandb/v1/client/TagAndValue.java b/src/main/java/org/apache/skywalking/banyandb/v1/client/TagAndValue.java
index afab5b6..32cd163 100644
--- a/src/main/java/org/apache/skywalking/banyandb/v1/client/TagAndValue.java
+++ b/src/main/java/org/apache/skywalking/banyandb/v1/client/TagAndValue.java
@@ -21,6 +21,7 @@ package org.apache.skywalking.banyandb.v1.client;
 import java.util.List;
 
 import com.google.protobuf.ByteString;
+import com.google.protobuf.NullValue;
 import lombok.EqualsAndHashCode;
 import org.apache.skywalking.banyandb.model.v1.BanyandbModel;
 
@@ -50,7 +51,21 @@ public abstract class TagAndValue<T> extends Value<T> {
         return this.value == null;
     }
 
-    static TagAndValue<?> build(BanyandbModel.Tag tag) {
+    /**
+     * TagValue is referenced from various derived data structs.
+     *
+     * @return TagValue to be included
+     */
+    protected abstract BanyandbModel.TagValue buildTypedTagValue();
+
+    public BanyandbModel.Tag build() {
+        return BanyandbModel.Tag.newBuilder()
+                .setKey(this.tagName)
+                .setValue(buildTypedTagValue())
+                .build();
+    }
+
+    public static TagAndValue<?> fromProtobuf(BanyandbModel.Tag tag) {
         switch (tag.getValue().getValueCase()) {
             case INT:
                 return new LongTagPair(tag.getKey(), tag.getValue().getInt().getValue());
@@ -69,44 +84,124 @@ public abstract class TagAndValue<T> extends Value<T> {
         }
     }
 
+    @EqualsAndHashCode(callSuper = true)
     public static class StringTagPair extends TagAndValue<String> {
         StringTagPair(final String tagName, final String value) {
             super(tagName, value);
         }
+
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setStr(BanyandbModel.Str
+                            .newBuilder()
+                            .setValue(value).build())
+                    .build();
+        }
+    }
+
+    public static TagAndValue<String> newStringTag(final String tagName, final String value) {
+        return new StringTagPair(tagName, value);
     }
 
+    @EqualsAndHashCode(callSuper = true)
     public static class StringArrayTagPair extends TagAndValue<List<String>> {
         StringArrayTagPair(final String tagName, final List<String> value) {
             super(tagName, value);
         }
+
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setStrArray(BanyandbModel.StrArray
+                            .newBuilder()
+                            .addAllValue(value).build())
+                    .build();
+        }
     }
 
+    public static TagAndValue<List<String>> newStringArrayTagPair(final String tagName, final List<String> value) {
+        return new StringArrayTagPair(tagName, value);
+    }
+
+    @EqualsAndHashCode(callSuper = true)
     public static class LongTagPair extends TagAndValue<Long> {
         LongTagPair(final String tagName, final Long value) {
             super(tagName, value);
         }
+
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setInt(BanyandbModel.Int
+                            .newBuilder()
+                            .setValue(value).build())
+                    .build();
+        }
     }
 
+    public static TagAndValue<Long> newLongTag(final String tagName, final long value) {
+        return new LongTagPair(tagName, value);
+    }
+
+    @EqualsAndHashCode(callSuper = true)
     public static class LongArrayTagPair extends TagAndValue<List<Long>> {
         LongArrayTagPair(final String tagName, final List<Long> value) {
             super(tagName, value);
         }
+
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setIntArray(BanyandbModel.IntArray
+                            .newBuilder()
+                            .addAllValue(value).build())
+                    .build();
+        }
     }
 
+    public static TagAndValue<Long> newLongArrayTag(final String tagName, final long value) {
+        return new LongTagPair(tagName, value);
+    }
+
+    @EqualsAndHashCode(callSuper = true)
     public static class BinaryTagPair extends TagAndValue<ByteString> {
         public BinaryTagPair(String fieldName, ByteString byteString) {
             super(fieldName, byteString);
         }
+
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setBinaryData(this.value)
+                    .build();
+        }
+    }
+
+    public static TagAndValue<ByteString> newBinaryTag(final String tagName, final byte[] bytes) {
+        return new BinaryTagPair(tagName, ByteString.copyFrom(bytes));
     }
 
+    @EqualsAndHashCode(callSuper = true)
     public static class NullTagPair extends TagAndValue<Void> {
         NullTagPair(final String tagName) {
             super(tagName, null);
         }
 
+        @Override
+        protected BanyandbModel.TagValue buildTypedTagValue() {
+            return BanyandbModel.TagValue.newBuilder()
+                    .setNull(NullValue.NULL_VALUE)
+                    .build();
+        }
+
         @Override
         public boolean isNull() {
             return true;
         }
     }
+
+    public static TagAndValue<Void> newNullTag(final String tagName) {
+        return new NullTagPair(tagName);
+    }
 }
diff --git a/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/Property.java b/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/Property.java
new file mode 100644
index 0000000..785389b
--- /dev/null
+++ b/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/Property.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the 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.
+ * The 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 org.apache.skywalking.banyandb.v1.client.metadata;
+
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.model.v1.BanyandbModel;
+import org.apache.skywalking.banyandb.property.v1.BanyandbProperty;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.banyandb.v1.client.util.TimeUtils;
+
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.List;
+
+@AutoValue
+public abstract class Property extends NamedSchema<BanyandbProperty.Property> {
+    public abstract String id();
+
+    abstract ImmutableList<TagAndValue<?>> tags();
+
+    @Override
+    public BanyandbProperty.Property serialize() {
+        List<BanyandbModel.Tag> tags = new ArrayList<>(this.tags().size());
+        for (final TagAndValue<?> tagAndValue : this.tags()) {
+            tags.add(tagAndValue.build());
+        }
+        return BanyandbProperty.Property.newBuilder()
+                .setMetadata(BanyandbProperty.Metadata.newBuilder()
+                        .setId(id())
+                        .setContainer(buildMetadata())
+                        .build())
+                .addAllTags(tags)
+                .build();
+    }
+
+    public static Builder create(String group, String name, String id) {
+        return new AutoValue_Property.Builder().setGroup(group).setName(name).setId(id);
+    }
+
+    static Property fromProtobuf(BanyandbProperty.Property pb) {
+        final Property.Builder b = Property.create(pb.getMetadata().getContainer().getGroup(),
+                        pb.getMetadata().getContainer().getName(),
+                        pb.getMetadata().getId())
+                .setUpdatedAt(TimeUtils.parseTimestamp(pb.getUpdatedAt()));
+
+        // build tag family spec
+        for (int i = 0; i < pb.getTagsCount(); i++) {
+            b.addTag(TagAndValue.fromProtobuf(pb.getTags(i)));
+        }
+
+        return b.build();
+    }
+
+    @AutoValue.Builder
+    public abstract static class Builder {
+        abstract Builder setGroup(String group);
+
+        abstract Builder setName(String name);
+
+        abstract Builder setUpdatedAt(ZonedDateTime updatedAt);
+
+        public abstract Builder setId(String id);
+
+        abstract ImmutableList.Builder<TagAndValue<?>> tagsBuilder();
+
+        public final Builder addTag(TagAndValue<?> tagAndValue) {
+            tagsBuilder().add(tagAndValue);
+            return this;
+        }
+
+        public abstract Property build();
+    }
+}
diff --git a/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStore.java b/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStore.java
new file mode 100644
index 0000000..4023c24
--- /dev/null
+++ b/src/main/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStore.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the 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.
+ * The 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 org.apache.skywalking.banyandb.v1.client.metadata;
+
+import io.grpc.Channel;
+import org.apache.skywalking.banyandb.common.v1.BanyandbCommon;
+import org.apache.skywalking.banyandb.property.v1.BanyandbProperty;
+import org.apache.skywalking.banyandb.property.v1.PropertyServiceGrpc;
+import org.apache.skywalking.banyandb.v1.client.grpc.HandleExceptionsWith;
+import org.apache.skywalking.banyandb.v1.client.grpc.exception.BanyanDBException;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class PropertyStore {
+    private final PropertyServiceGrpc.PropertyServiceBlockingStub stub;
+
+    public PropertyStore(Channel channel) {
+        this.stub = PropertyServiceGrpc.newBlockingStub(channel);
+    }
+
+    public void create(Property payload) throws BanyanDBException {
+        HandleExceptionsWith.callAndTranslateApiException(() ->
+                this.stub.create(BanyandbProperty.CreateRequest.newBuilder()
+                        .setProperty(payload.serialize())
+                        .build()));
+    }
+
+    public void update(Property payload) throws BanyanDBException {
+        HandleExceptionsWith.callAndTranslateApiException(() ->
+                this.stub.update(BanyandbProperty.UpdateRequest.newBuilder()
+                        .setProperty(payload.serialize())
+                        .build()));
+    }
+
+    public boolean delete(String group, String name, String id) throws BanyanDBException {
+        BanyandbProperty.DeleteResponse resp = HandleExceptionsWith.callAndTranslateApiException(() ->
+                this.stub.delete(BanyandbProperty.DeleteRequest.newBuilder()
+                        .setMetadata(BanyandbProperty.Metadata
+                                .newBuilder()
+                                .setContainer(BanyandbCommon.Metadata.newBuilder()
+                                        .setGroup(group)
+                                        .setName(name)
+                                        .build())
+                                .setId(id)
+                                .build())
+                        .build()));
+        return resp != null && resp.getDeleted();
+    }
+
+    public Property get(String group, String name, String id) throws BanyanDBException {
+        BanyandbProperty.GetResponse resp = HandleExceptionsWith.callAndTranslateApiException(() ->
+                this.stub.get(BanyandbProperty.GetRequest.newBuilder()
+                        .setMetadata(BanyandbProperty.Metadata
+                                .newBuilder()
+                                .setContainer(BanyandbCommon.Metadata.newBuilder()
+                                        .setGroup(group)
+                                        .setName(name)
+                                        .build())
+                                .setId(id)
+                                .build())
+                        .build()));
+
+        return Property.fromProtobuf(resp.getProperty());
+    }
+
+    public List<Property> list(String group, String name) throws BanyanDBException {
+        BanyandbProperty.ListResponse resp = HandleExceptionsWith.callAndTranslateApiException(() ->
+                this.stub.list(BanyandbProperty.ListRequest.newBuilder()
+                        .setContainer(BanyandbCommon.Metadata.newBuilder()
+                                .setGroup(group)
+                                .setName(name)
+                                .build())
+                        .build()));
+
+        return resp.getPropertyList().stream().map(Property::fromProtobuf).collect(Collectors.toList());
+    }
+}
diff --git a/src/main/proto/banyandb/v1/banyandb-property.proto b/src/main/proto/banyandb/v1/banyandb-property.proto
new file mode 100644
index 0000000..fafe19f
--- /dev/null
+++ b/src/main/proto/banyandb/v1/banyandb-property.proto
@@ -0,0 +1,90 @@
+// 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.
+
+syntax = "proto3";
+
+option java_package = "org.apache.skywalking.banyandb.property.v1";
+
+package banyandb.property.v1;
+
+import "google/protobuf/timestamp.proto";
+import "banyandb/v1/banyandb-common.proto";
+import "banyandb/v1/banyandb-model.proto";
+
+// Metadata is for multi-tenant use
+message Metadata {
+  // container is created when it receives the first property
+  common.v1.Metadata container = 1;
+  // id identifies a property
+  string id = 2;
+}
+
+// Property stores the user defined data
+message Property {
+  // metadata is the identity of a property
+  Metadata metadata = 1;
+  // tag stores the content of a property
+  repeated model.v1.Tag tags = 2;
+  // updated_at indicates when the property is updated
+  google.protobuf.Timestamp updated_at = 3;
+}
+
+message CreateRequest {
+  banyandb.property.v1.Property property = 1;
+}
+
+message CreateResponse {
+}
+
+message UpdateRequest {
+  banyandb.property.v1.Property property = 1;
+}
+
+message UpdateResponse {
+}
+
+message DeleteRequest {
+  banyandb.property.v1.Metadata metadata = 1;
+}
+
+message DeleteResponse {
+  bool deleted = 1;
+}
+
+message GetRequest {
+  banyandb.property.v1.Metadata metadata = 1;
+}
+
+message GetResponse {
+  banyandb.property.v1.Property property = 1;
+}
+
+message ListRequest {
+  banyandb.common.v1.Metadata container = 1;
+}
+
+message ListResponse {
+  repeated banyandb.property.v1.Property property = 1;
+}
+
+service PropertyService {
+  rpc Create(CreateRequest) returns (CreateResponse);
+  rpc Update(UpdateRequest) returns (UpdateResponse);
+  rpc Delete(DeleteRequest) returns (DeleteResponse);
+  rpc Get(GetRequest) returns (GetResponse);
+  rpc List(ListRequest) returns (ListResponse);
+}
diff --git a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClientTestCI.java b/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClientTestCI.java
index 93b277a..48e43ee 100644
--- a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClientTestCI.java
+++ b/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBClientTestCI.java
@@ -30,11 +30,11 @@ import java.io.IOException;
 public class BanyanDBClientTestCI {
     private static final String REGISTRY = "ghcr.io";
     private static final String IMAGE_NAME = "apache/skywalking-banyandb";
-    private static final String TAG = "d061ab4abe0232c868f60cd3f311877b5a3703ac";
+    private static final String TAG = "ad94bb9e39276a985d1647cacae4afc565b17d83";
 
     private static final String IMAGE = REGISTRY + "/" + IMAGE_NAME + ":" + TAG;
 
-    private static final int BANYANDB_PORT = 17912;
+    protected static final int BANYANDB_PORT = 17912;
 
     @Rule
     public GenericContainer<?> banyanDB = new GenericContainer<>(
diff --git a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBMeasureQueryIntegrationTests.java b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBMeasureQueryTests.java
similarity index 98%
rename from src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBMeasureQueryIntegrationTests.java
rename to src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBMeasureQueryTests.java
index c03c9ed..740335d 100644
--- a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBMeasureQueryIntegrationTests.java
+++ b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBMeasureQueryTests.java
@@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.awaitility.Awaitility.await;
 
-public class BanyanDBMeasureQueryIntegrationTests extends BanyanDBClientTestCI {
+public class ITBanyanDBMeasureQueryTests extends BanyanDBClientTestCI {
     private MeasureBulkWriteProcessor processor;
 
     @Before
diff --git a/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBPropertyTests.java b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBPropertyTests.java
new file mode 100644
index 0000000..08b8d41
--- /dev/null
+++ b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBPropertyTests.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the 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.
+ * The 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 org.apache.skywalking.banyandb.v1.client;
+
+import io.grpc.Status;
+import org.apache.skywalking.banyandb.v1.client.grpc.exception.BanyanDBException;
+import org.apache.skywalking.banyandb.v1.client.metadata.Catalog;
+import org.apache.skywalking.banyandb.v1.client.metadata.Duration;
+import org.apache.skywalking.banyandb.v1.client.metadata.Group;
+import org.apache.skywalking.banyandb.v1.client.metadata.Property;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import static org.awaitility.Awaitility.await;
+
+public class ITBanyanDBPropertyTests extends BanyanDBClientTestCI {
+    @Before
+    public void setUp() throws IOException, BanyanDBException, InterruptedException {
+        super.setUpConnection();
+        Group expectedGroup = this.client.define(
+                Group.create("default", Catalog.STREAM, 2, 0, Duration.ofDays(7))
+        );
+        Assert.assertNotNull(expectedGroup);
+    }
+
+    @After
+    public void tearDown() throws IOException {
+        this.closeClient();
+    }
+
+    @Test
+    public void test_PropertyCreateAndGet() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.client.save(property);
+
+        await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> {
+            Property gotProperty = client.findProperty("default", "sw", "ui_template");
+            Assert.assertNotNull(gotProperty);
+            Assert.assertEquals(property, gotProperty);
+        });
+    }
+
+    @Test
+    public void test_PropertyCreateDeleteAndGet() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.client.save(property);
+
+        await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> {
+            Property gotProperty = client.findProperty("default", "sw", "ui_template");
+            Assert.assertNotNull(gotProperty);
+            Assert.assertEquals(property, gotProperty);
+        });
+
+        Assert.assertTrue(this.client.deleteProperty("default", "sw", "ui_template"));
+
+        try {
+            client.findProperty("default", "sw", "ui_template");
+            Assert.fail();
+        } catch (BanyanDBException ex) {
+            Assert.assertEquals(Status.Code.NOT_FOUND, ex.getStatus());
+        }
+    }
+
+    @Test
+    public void test_PropertyCreateUpdateAndGet() throws BanyanDBException {
+        Property property1 = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.client.save(property1);
+
+        Property property2 = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "world"))
+                .build();
+        this.client.save(property2);
+
+        await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> {
+            Property gotProperty = client.findProperty("default", "sw", "ui_template");
+            Assert.assertNotNull(gotProperty);
+            Assert.assertEquals(property2, gotProperty);
+        });
+    }
+}
diff --git a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBStreamQueryIntegrationTests.java b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBStreamQueryTests.java
similarity index 98%
rename from src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBStreamQueryIntegrationTests.java
rename to src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBStreamQueryTests.java
index 0c1034f..90e8079 100644
--- a/src/test/java/org/apache/skywalking/banyandb/v1/client/BanyanDBStreamQueryIntegrationTests.java
+++ b/src/test/java/org/apache/skywalking/banyandb/v1/client/ITBanyanDBStreamQueryTests.java
@@ -37,7 +37,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.awaitility.Awaitility.await;
 
-public class BanyanDBStreamQueryIntegrationTests extends BanyanDBClientTestCI {
+public class ITBanyanDBStreamQueryTests extends BanyanDBClientTestCI {
     private StreamBulkWriteProcessor processor;
 
     @Before
diff --git a/src/test/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStoreTest.java b/src/test/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStoreTest.java
new file mode 100644
index 0000000..062c932
--- /dev/null
+++ b/src/test/java/org/apache/skywalking/banyandb/v1/client/metadata/PropertyStoreTest.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the 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.
+ * The 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 org.apache.skywalking.banyandb.v1.client.metadata;
+
+import io.grpc.stub.StreamObserver;
+import org.apache.skywalking.banyandb.property.v1.BanyandbProperty;
+import org.apache.skywalking.banyandb.property.v1.PropertyServiceGrpc;
+import org.apache.skywalking.banyandb.v1.client.AbstractBanyanDBClientTest;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.banyandb.v1.client.grpc.exception.BanyanDBException;
+import org.apache.skywalking.banyandb.v1.client.util.TimeUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.io.IOException;
+import java.time.ZonedDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.mockito.AdditionalAnswers.delegatesTo;
+import static org.powermock.api.mockito.PowerMockito.mock;
+
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
+public class PropertyStoreTest extends AbstractBanyanDBClientTest {
+    private PropertyStore store;
+
+    private Map<String, BanyandbProperty.Property> memory;
+
+    private final PropertyServiceGrpc.PropertyServiceImplBase propertyServiceImpl = mock(PropertyServiceGrpc.PropertyServiceImplBase.class, delegatesTo(
+            new PropertyServiceGrpc.PropertyServiceImplBase() {
+                @Override
+                public void create(BanyandbProperty.CreateRequest request, StreamObserver<BanyandbProperty.CreateResponse> responseObserver) {
+                    BanyandbProperty.Property p = request.getProperty().toBuilder()
+                            .setUpdatedAt(TimeUtils.buildTimestamp(ZonedDateTime.now()))
+                            .build();
+                    memory.put(format(p.getMetadata()), p);
+                    responseObserver.onNext(BanyandbProperty.CreateResponse.newBuilder().build());
+                    responseObserver.onCompleted();
+                }
+
+                @Override
+                public void update(BanyandbProperty.UpdateRequest request, StreamObserver<BanyandbProperty.UpdateResponse> responseObserver) {
+                    BanyandbProperty.Property p = request.getProperty().toBuilder()
+                            .setUpdatedAt(TimeUtils.buildTimestamp(ZonedDateTime.now()))
+                            .build();
+                    memory.put(format(p.getMetadata()), p);
+                    responseObserver.onNext(BanyandbProperty.UpdateResponse.newBuilder().build());
+                    responseObserver.onCompleted();
+                }
+
+                @Override
+                public void delete(BanyandbProperty.DeleteRequest request, StreamObserver<BanyandbProperty.DeleteResponse> responseObserver) {
+                    final BanyandbProperty.Property p = memory.remove(format(request.getMetadata()));
+                    responseObserver.onNext(BanyandbProperty.DeleteResponse.newBuilder().setDeleted(p != null).build());
+                    responseObserver.onCompleted();
+                }
+
+                @Override
+                public void get(BanyandbProperty.GetRequest request, StreamObserver<BanyandbProperty.GetResponse> responseObserver) {
+                    final BanyandbProperty.Property p = memory.get(format(request.getMetadata()));
+                    responseObserver.onNext(BanyandbProperty.GetResponse.newBuilder().setProperty(p).build());
+                    responseObserver.onCompleted();
+                }
+
+                @Override
+                public void list(BanyandbProperty.ListRequest request, StreamObserver<BanyandbProperty.ListResponse> responseObserver) {
+                    responseObserver.onNext(BanyandbProperty.ListResponse.newBuilder().addAllProperty(memory.values()).build());
+                    responseObserver.onCompleted();
+                }
+            }));
+
+    @Before
+    public void setUp() throws IOException {
+        super.setUp(bindService(propertyServiceImpl));
+        this.memory = new HashMap<>();
+        this.store = new PropertyStore(this.channel);
+    }
+
+    @Test
+    public void testPropertyStore_create() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.store.create(property);
+        Assert.assertEquals(memory.size(), 1);
+    }
+
+    @Test
+    public void testPropertyStore_createAndGet() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.store.create(property);
+        Property gotProperty = this.store.get("default", "sw", "ui_template");
+        Assert.assertNotNull(gotProperty);
+        Assert.assertEquals(property, gotProperty);
+        Assert.assertNotNull(gotProperty.updatedAt());
+    }
+
+    @Test
+    public void testPropertyStore_createAndList() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.store.create(property);
+        List<Property> listProperties = this.store.list("default", "sw");
+        Assert.assertNotNull(listProperties);
+        Assert.assertEquals(1, listProperties.size());
+        Assert.assertEquals(listProperties.get(0), property);
+    }
+
+    @Test
+    public void testPropertyStore_createAndDelete() throws BanyanDBException {
+        Property property = Property.create("default", "sw", "ui_template")
+                .addTag(TagAndValue.newStringTag("name", "hello"))
+                .build();
+        this.store.create(property);
+        boolean deleted = this.store.delete("default", "sw", "ui_template");
+        Assert.assertTrue(deleted);
+        Assert.assertEquals(0, memory.size());
+    }
+
+    static String format(BanyandbProperty.Metadata metadata) {
+        return metadata.getContainer().getGroup() + ":" + metadata.getContainer().getName() + "/" + metadata.getId();
+    }
+}