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/30 09:03:25 UTC

[skywalking] 06/24: complete basic logic

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

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

commit 799608dd59a61c28827560c3086a8ef8cbaa13cd
Author: Megrez Lu <lu...@gmail.com>
AuthorDate: Sun Dec 5 22:44:22 2021 +0800

    complete basic logic
---
 .../storage/plugin/banyandb/BanyanDBSchema.java    |  66 ------------
 .../plugin/banyandb/BanyanDBStorageProvider.java   |   6 +-
 .../deserializer/AbstractBanyanDBDeserializer.java |  20 +++-
 ...geMapper.java => AlarmMessageDeserializer.java} |  22 +++-
 .../deserializer/BanyanDBDeserializerFactory.java  |  48 ++++++---
 ...raceMapper.java => BasicTraceDeserializer.java} |  22 +++-
 ...apper.java => BrowserErrorLogDeserializer.java} |  27 ++++-
 ...ava => DashboardConfigurationDeserializer.java} |  22 +++-
 .../deserializer/DatabaseDeserializer.java         |  43 ++++++++
 .../banyandb/deserializer/DatabaseMapper.java      |  25 -----
 .../deserializer/EndpointDeserializer.java         |  43 ++++++++
 .../banyandb/deserializer/EndpointMapper.java      |  25 -----
 .../{EventMapper.java => EventDeserializer.java}   |  22 +++-
 .../{LogMapper.java => LogDeserializer.java}       |  26 ++++-
 ...r.java => NetworkAddressAliasDeserializer.java} |  22 +++-
 ...askMapper.java => ProfileTaskDeserializer.java} |  22 +++-
 ...Mapper.java => ProfileTaskLogDeserializer.java} |  22 +++-
 ...> ProfileThreadSnapshotRecordDeserializer.java} |  22 +++-
 .../RowEntityDeserializer.java}                    |  17 ++-
 .../banyandb/deserializer/RowEntityMapper.java     |   8 --
 ...dMapper.java => SegmentRecordDeserializer.java} |  22 +++-
 ...ServiceMapper.java => ServiceDeserializer.java} |  22 +++-
 .../deserializer/ServiceInstanceDeserializer.java  |  79 ++++++++++++++
 .../deserializer/ServiceInstanceMapper.java        |  70 ------------
 .../plugin/banyandb/schema/AlarmRecordBuilder.java |  55 ++++++++++
 .../BanyanDBMetricsBuilder.java}                   |  21 ++--
 .../banyandb/schema/BanyanDBRecordBuilder.java     |  61 +++++++++++
 .../schema/BanyanDBStorageDataBuilder.java         |  52 +++++++++
 .../schema/BrowserErrorLogRecordBuilder.java       |  46 ++++++++
 .../plugin/banyandb/schema/EventBuilder.java       |  41 ++++++++
 .../plugin/banyandb/schema/LogRecordBuilder.java   |  58 ++++++++++
 .../storage/plugin/banyandb/schema/Metadata.java   |  70 ++++++++++++
 .../schema/NetworkAddressAliasBuilder.java         |  44 ++++++++
 .../schema/ProfileTaskLogRecordBuilder.java        |  44 ++++++++
 .../banyandb/schema/ProfileTaskRecordBuilder.java  |  44 ++++++++
 .../schema/ProfileThreadSnapshotRecordBuilder.java |  45 ++++++++
 .../banyandb/schema/SegmentRecordBuilder.java      |  60 +++++++++++
 .../plugin/banyandb/schema/UITemplateBuilder.java  |  55 ++++++++++
 .../banyandb/stream/AbstractBanyanDBDAO.java       |  18 ++++
 .../banyandb/stream/BanyanDBAlarmQueryDAO.java     |  29 ++++-
 .../stream/BanyanDBBrowserLogQueryDAO.java         |  36 +++++--
 .../banyandb/stream/BanyanDBEventQueryDAO.java     |  19 +++-
 .../banyandb/stream/BanyanDBLogQueryDAO.java       |  49 ++++++---
 ...sertRequest.java => BanyanDBManagementDAO.java} |  22 ++--
 .../banyandb/stream/BanyanDBMetadataQueryDAO.java  |  18 ++++
 .../plugin/banyandb/stream/BanyanDBMetricsDAO.java |  53 ++++++++++
 .../stream/BanyanDBNetworkAddressAliasDAO.java     |  18 ++++
 ...sertRequest.java => BanyanDBNoneStreamDAO.java} |  22 ++--
 .../stream/BanyanDBProfileTaskLogQueryDAO.java     |  22 +++-
 .../stream/BanyanDBProfileTaskQueryDAO.java        |  25 ++++-
 .../BanyanDBProfileThreadSnapshotQueryDAO.java     |  26 +++--
 .../plugin/banyandb/stream/BanyanDBRecordDAO.java  |  48 +--------
 .../stream/BanyanDBSegmentRecordBuilder.java       |  58 ----------
 .../plugin/banyandb/stream/BanyanDBStorageDAO.java | 117 +++++++++++++++------
 .../stream/BanyanDBStreamInsertRequest.java        |   1 -
 .../banyandb/stream/BanyanDBTraceQueryDAO.java     |  32 +++---
 .../stream/BanyanDBUITemplateManagementDAO.java    |  23 +++-
 57 files changed, 1588 insertions(+), 467 deletions(-)

diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBSchema.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBSchema.java
deleted file mode 100644
index 3ebf87776b..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBSchema.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.oap.server.storage.plugin.banyandb;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import lombok.Getter;
-
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-public class BanyanDBSchema {
-    public static final String NAME = "sw";
-    public static final String GROUP = "default";
-    public static final List<String> FIELD_NAMES;
-
-    public static final Set<String> INDEX_FIELDS = ImmutableSet.of("http.method", "status_code", "db.type",
-            "db.instance", "mq.queue", "mq.topic", "mq.broker");
-
-    static {
-        Set<String> fields = new LinkedHashSet<>();
-        fields.add("trace_id");
-        fields.add("state");
-        fields.add("service_id");
-        fields.add("service_instance_id");
-        fields.add("endpoint_id");
-        fields.add("duration");
-        fields.add("start_time");
-        fields.add("http.method");
-        fields.add("status_code");
-        fields.add("db.type");
-        fields.add("db.instance");
-        fields.add("mq.queue");
-        fields.add("mq.topic");
-        fields.add("mq.broker");
-        FIELD_NAMES = ImmutableList.copyOf(fields);
-    }
-
-    public enum TraceState {
-        ALL(0), SUCCESS(1), ERROR(2);
-
-        @Getter
-        private final int state;
-
-        TraceState(int state) {
-            this.state = state;
-        }
-    }
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBStorageProvider.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBStorageProvider.java
index c4aa1016ca..09b72509a8 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBStorageProvider.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/BanyanDBStorageProvider.java
@@ -22,7 +22,11 @@ import org.apache.skywalking.oap.server.core.CoreModule;
 import org.apache.skywalking.oap.server.core.config.ConfigService;
 import org.apache.skywalking.oap.server.core.storage.StorageBuilderFactory;
 import org.apache.skywalking.oap.server.core.storage.StorageModule;
-import org.apache.skywalking.oap.server.library.module.*;
+import org.apache.skywalking.oap.server.library.module.ModuleConfig;
+import org.apache.skywalking.oap.server.library.module.ModuleDefine;
+import org.apache.skywalking.oap.server.library.module.ModuleProvider;
+import org.apache.skywalking.oap.server.library.module.ModuleStartException;
+import org.apache.skywalking.oap.server.library.module.ServiceNotProvidedException;
 import org.apache.skywalking.oap.server.telemetry.TelemetryModule;
 import org.apache.skywalking.oap.server.telemetry.api.HealthCheckMetrics;
 import org.apache.skywalking.oap.server.telemetry.api.MetricsCreator;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AbstractBanyanDBDeserializer.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AbstractBanyanDBDeserializer.java
index 44adcb8869..0b2f0644ad 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AbstractBanyanDBDeserializer.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AbstractBanyanDBDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import org.apache.skywalking.banyandb.v1.client.StreamQuery;
@@ -6,7 +24,7 @@ import org.apache.skywalking.banyandb.v1.client.TimestampRange;
 import java.util.Collections;
 import java.util.List;
 
-public abstract class AbstractBanyanDBDeserializer<T> implements RowEntityMapper<T> {
+public abstract class AbstractBanyanDBDeserializer<T> implements RowEntityDeserializer<T> {
     private final String indexName;
     private final List<String> searchableProjection;
     private final List<String> dataProjection;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageDeserializer.java
similarity index 71%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageDeserializer.java
index 939496e061..3165d60afa 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/AlarmMessageDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.base.Charsets;
@@ -15,10 +33,10 @@ import org.apache.skywalking.oap.server.core.query.type.KeyValue;
 
 import java.util.List;
 
-public class AlarmMessageMapper extends AbstractBanyanDBDeserializer<AlarmMessage> {
+public class AlarmMessageDeserializer extends AbstractBanyanDBDeserializer<AlarmMessage> {
     private final Gson GSON = new Gson();
 
-    public AlarmMessageMapper() {
+    public AlarmMessageDeserializer() {
         super(AlarmRecord.INDEX_NAME,
                 ImmutableList.of(AlarmRecord.SCOPE, AlarmRecord.START_TIME),
                 ImmutableList.of(AlarmRecord.ID0, AlarmRecord.ID1, AlarmRecord.ALARM_MESSAGE, AlarmRecord.TAGS_RAW_DATA));
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BanyanDBDeserializerFactory.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BanyanDBDeserializerFactory.java
index dd4af41736..859812530e 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BanyanDBDeserializerFactory.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BanyanDBDeserializerFactory.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import org.apache.skywalking.oap.server.core.analysis.manual.networkalias.NetworkAddressAlias;
@@ -26,21 +44,21 @@ public enum BanyanDBDeserializerFactory {
 
     BanyanDBDeserializerFactory() {
         registry = new HashMap<>(10);
-        register(AlarmMessage.class, new AlarmMessageMapper());
-        register(BasicTrace.class, new BasicTraceMapper());
-        register(BrowserErrorLog.class, new BrowserErrorLogMapper());
-        register(DashboardConfiguration.class, new DashboardConfigurationMapper());
-        register(Database.class, new DatabaseMapper());
-        register(Endpoint.class, new EndpointMapper());
-        register(Event.class, new EventMapper());
-        register(Log.class, new LogMapper());
-        register(NetworkAddressAlias.class, new NetworkAddressAliasMapper());
-        register(ProfileTaskLog.class, new ProfileTaskLogMapper());
-        register(ProfileTask.class, new ProfileTaskMapper());
-        register(ProfileThreadSnapshotRecord.class, new ProfileThreadSnapshotRecordMapper());
-        register(SegmentRecord.class, new SegmentRecordMapper());
-        register(ServiceInstance.class, new ServiceInstanceMapper());
-        register(Service.class, new ServiceMapper());
+        register(AlarmMessage.class, new AlarmMessageDeserializer());
+        register(BasicTrace.class, new BasicTraceDeserializer());
+        register(BrowserErrorLog.class, new BrowserErrorLogDeserializer());
+        register(DashboardConfiguration.class, new DashboardConfigurationDeserializer());
+        register(Database.class, new DatabaseDeserializer());
+        register(Endpoint.class, new EndpointDeserializer());
+        register(Event.class, new EventDeserializer());
+        register(Log.class, new LogDeserializer());
+        register(NetworkAddressAlias.class, new NetworkAddressAliasDeserializer());
+        register(ProfileTaskLog.class, new ProfileTaskLogDeserializer());
+        register(ProfileTask.class, new ProfileTaskDeserializer());
+        register(ProfileThreadSnapshotRecord.class, new ProfileThreadSnapshotRecordDeserializer());
+        register(SegmentRecord.class, new SegmentRecordDeserializer());
+        register(ServiceInstance.class, new ServiceInstanceDeserializer());
+        register(Service.class, new ServiceDeserializer());
     }
 
     private <T> void register(Class<T> clazz, AbstractBanyanDBDeserializer<T> mapper) {
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceDeserializer.java
similarity index 58%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceDeserializer.java
index 5d7ae36590..949ae43e16 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BasicTraceDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -9,8 +27,8 @@ import org.apache.skywalking.oap.server.core.query.type.BasicTrace;
 
 import java.util.List;
 
-public class BasicTraceMapper extends AbstractBanyanDBDeserializer<BasicTrace> {
-    public BasicTraceMapper() {
+public class BasicTraceDeserializer extends AbstractBanyanDBDeserializer<BasicTrace> {
+    public BasicTraceDeserializer() {
         super(SegmentRecord.INDEX_NAME, ImmutableList.of("trace_id", "state", "endpoint_id", "duration", "start_time"));
     }
 
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogDeserializer.java
similarity index 70%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogDeserializer.java
index 0189a77049..92605afa67 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/BrowserErrorLogDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -12,14 +30,13 @@ import org.apache.skywalking.oap.server.core.query.type.ErrorCategory;
 import java.util.Collections;
 import java.util.List;
 
-public class BrowserErrorLogMapper extends AbstractBanyanDBDeserializer<BrowserErrorLog> {
-    public BrowserErrorLogMapper() {
+public class BrowserErrorLogDeserializer extends AbstractBanyanDBDeserializer<BrowserErrorLog> {
+    public BrowserErrorLogDeserializer() {
         super(BrowserErrorLogRecord.INDEX_NAME,
                 ImmutableList.of(BrowserErrorLogRecord.SERVICE_ID,
                         BrowserErrorLogRecord.SERVICE_VERSION_ID,
                         BrowserErrorLogRecord.PAGE_PATH_ID,
-                        BrowserErrorLogRecord.ERROR_CATEGORY,
-                        BrowserErrorLogRecord.TIMESTAMP),
+                        BrowserErrorLogRecord.ERROR_CATEGORY),
                 Collections.singletonList(BrowserErrorLogRecord.DATA_BINARY));
     }
 
@@ -32,7 +49,7 @@ public class BrowserErrorLogMapper extends AbstractBanyanDBDeserializer<BrowserE
         log.setServiceVersion((String) searchable.get(1).getValue());
         log.setPagePath((String) searchable.get(2).getValue());
         log.setCategory(ErrorCategory.valueOf((String) searchable.get(3).getValue()));
-        log.setTime(((Number) searchable.get(4).getValue()).longValue());
+        log.setTime(row.getTimestamp());
         final List<TagAndValue<?>> data = row.getTagFamilies().get(1);
         Object o = data.get(0).getValue();
         if (o instanceof ByteString && !((ByteString) o).isEmpty()) {
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationDeserializer.java
similarity index 64%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationDeserializer.java
index cb6b4f3c2f..c46db9df8d 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DashboardConfigurationDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -10,8 +28,8 @@ import org.apache.skywalking.oap.server.library.util.BooleanUtils;
 
 import java.util.List;
 
-public class DashboardConfigurationMapper extends AbstractBanyanDBDeserializer<DashboardConfiguration> {
-    public DashboardConfigurationMapper() {
+public class DashboardConfigurationDeserializer extends AbstractBanyanDBDeserializer<DashboardConfiguration> {
+    public DashboardConfigurationDeserializer() {
         super(UITemplate.INDEX_NAME,
                 ImmutableList.of(UITemplate.NAME, UITemplate.DISABLED),
                 ImmutableList.of(UITemplate.ACTIVATED, UITemplate.CONFIGURATION, UITemplate.TYPE));
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseDeserializer.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseDeserializer.java
new file mode 100644
index 0000000000..db6d875358
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseDeserializer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.client.RowEntity;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.service.ServiceTraffic;
+import org.apache.skywalking.oap.server.core.query.type.Database;
+
+import java.util.List;
+
+public class DatabaseDeserializer extends AbstractBanyanDBDeserializer<Database> {
+    public DatabaseDeserializer() {
+        super(ServiceTraffic.INDEX_NAME,
+                ImmutableList.of(ServiceTraffic.NAME, ServiceTraffic.NODE_TYPE));
+    }
+
+    @Override
+    public Database map(RowEntity row) {
+        Database database = new Database();
+        final List<TagAndValue<?>> searchable = row.getTagFamilies().get(0);
+        database.setId(row.getId());
+        database.setName((String) searchable.get(0).getValue());
+        return database;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseMapper.java
deleted file mode 100644
index 1c5a15f774..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/DatabaseMapper.java
+++ /dev/null
@@ -1,25 +0,0 @@
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.skywalking.banyandb.v1.client.RowEntity;
-import org.apache.skywalking.banyandb.v1.client.TagAndValue;
-import org.apache.skywalking.oap.server.core.analysis.manual.service.ServiceTraffic;
-import org.apache.skywalking.oap.server.core.query.type.Database;
-
-import java.util.List;
-
-public class DatabaseMapper extends AbstractBanyanDBDeserializer<Database> {
-    public DatabaseMapper() {
-        super(ServiceTraffic.INDEX_NAME,
-                ImmutableList.of(ServiceTraffic.NAME, ServiceTraffic.NODE_TYPE));
-    }
-
-    @Override
-    public Database map(RowEntity row) {
-        Database database = new Database();
-        final List<TagAndValue<?>> searchable = row.getTagFamilies().get(0);
-        database.setId(row.getId());
-        database.setName((String) searchable.get(0).getValue());
-        return database;
-    }
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointDeserializer.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointDeserializer.java
new file mode 100644
index 0000000000..e86867c771
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointDeserializer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.client.RowEntity;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.endpoint.EndpointTraffic;
+import org.apache.skywalking.oap.server.core.query.type.Endpoint;
+
+import java.util.List;
+
+public class EndpointDeserializer extends AbstractBanyanDBDeserializer<Endpoint> {
+    public EndpointDeserializer() {
+        super(EndpointTraffic.INDEX_NAME,
+                ImmutableList.of(EndpointTraffic.NAME, EndpointTraffic.SERVICE_ID));
+    }
+
+    @Override
+    public Endpoint map(RowEntity row) {
+        Endpoint endpoint = new Endpoint();
+        final List<TagAndValue<?>> searchable = row.getTagFamilies().get(0);
+        endpoint.setName((String) searchable.get(0).getValue());
+        endpoint.setId((String) searchable.get(1).getValue());
+        return endpoint;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointMapper.java
deleted file mode 100644
index 8f81df039c..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EndpointMapper.java
+++ /dev/null
@@ -1,25 +0,0 @@
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.skywalking.banyandb.v1.client.RowEntity;
-import org.apache.skywalking.banyandb.v1.client.TagAndValue;
-import org.apache.skywalking.oap.server.core.analysis.manual.endpoint.EndpointTraffic;
-import org.apache.skywalking.oap.server.core.query.type.Endpoint;
-
-import java.util.List;
-
-public class EndpointMapper extends AbstractBanyanDBDeserializer<Endpoint> {
-    public EndpointMapper() {
-        super(EndpointTraffic.INDEX_NAME,
-                ImmutableList.of(EndpointTraffic.NAME, EndpointTraffic.SERVICE_ID));
-    }
-
-    @Override
-    public Endpoint map(RowEntity row) {
-        Endpoint endpoint = new Endpoint();
-        final List<TagAndValue<?>> searchable = row.getTagFamilies().get(0);
-        endpoint.setName((String) searchable.get(0).getValue());
-        endpoint.setId((String) searchable.get(1).getValue());
-        return endpoint;
-    }
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventDeserializer.java
similarity index 66%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventDeserializer.java
index 63c9dc802c..39783228d2 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/EventDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -9,8 +27,8 @@ import org.apache.skywalking.oap.server.core.source.Event;
 
 import java.util.List;
 
-public class EventMapper extends AbstractBanyanDBDeserializer<org.apache.skywalking.oap.server.core.query.type.event.Event> {
-    public EventMapper() {
+public class EventDeserializer extends AbstractBanyanDBDeserializer<org.apache.skywalking.oap.server.core.query.type.event.Event> {
+    public EventDeserializer() {
         super(Event.INDEX_NAME,
                 ImmutableList.of(Event.UUID, Event.SERVICE, Event.SERVICE_INSTANCE, Event.ENDPOINT, Event.NAME,
                         Event.TYPE, Event.START_TIME, Event.END_TIME),
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogDeserializer.java
similarity index 70%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogDeserializer.java
index 3f65a82cea..f1bb3bec5d 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/LogDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -14,12 +32,12 @@ import org.apache.skywalking.oap.server.core.query.type.Log;
 
 import java.util.List;
 
-public class LogMapper extends AbstractBanyanDBDeserializer<Log> {
-    public LogMapper() {
+public class LogDeserializer extends AbstractBanyanDBDeserializer<Log> {
+    public LogDeserializer() {
         super(LogRecord.INDEX_NAME, ImmutableList.of(
                         AbstractLogRecord.SERVICE_ID, AbstractLogRecord.SERVICE_INSTANCE_ID,
                         AbstractLogRecord.ENDPOINT_ID, AbstractLogRecord.TRACE_ID, AbstractLogRecord.TRACE_SEGMENT_ID,
-                        AbstractLogRecord.SPAN_ID, AbstractLogRecord.TIMESTAMP),
+                        AbstractLogRecord.SPAN_ID),
                 ImmutableList.of(AbstractLogRecord.CONTENT_TYPE, AbstractLogRecord.CONTENT, AbstractLogRecord.TAGS_RAW_DATA));
     }
 
@@ -31,7 +49,7 @@ public class LogMapper extends AbstractBanyanDBDeserializer<Log> {
         log.setServiceInstanceId((String) searchable.get(1).getValue());
         log.setEndpointId((String) searchable.get(2).getValue());
         log.setTraceId((String) searchable.get(3).getValue());
-        log.setTimestamp(((Number) searchable.get(6).getValue()).longValue());
+        log.setTimestamp(row.getTimestamp());
         final List<TagAndValue<?>> data = row.getTagFamilies().get(1);
         if (data.get(2).getValue() == null || ((ByteString) data.get(2).getValue()).isEmpty()) {
             log.setContent("");
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasDeserializer.java
similarity index 62%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasDeserializer.java
index 7c231df86a..6255a0ea40 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/NetworkAddressAliasDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -8,8 +26,8 @@ import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
 
 import java.util.List;
 
-public class NetworkAddressAliasMapper extends AbstractBanyanDBDeserializer<NetworkAddressAlias> {
-    public NetworkAddressAliasMapper() {
+public class NetworkAddressAliasDeserializer extends AbstractBanyanDBDeserializer<NetworkAddressAlias> {
+    public NetworkAddressAliasDeserializer() {
         super(NetworkAddressAlias.INDEX_NAME,
                 ImmutableList.of(NetworkAddressAlias.LAST_UPDATE_TIME_BUCKET),
                 ImmutableList.of(Metrics.TIME_BUCKET, "address", "represent_service_id", "represent_service_instance_id"));
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskDeserializer.java
similarity index 66%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskDeserializer.java
index 2626a91c7d..d3eddedec6 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -8,10 +26,10 @@ import org.apache.skywalking.oap.server.core.query.type.ProfileTask;
 
 import java.util.List;
 
-public class ProfileTaskMapper extends AbstractBanyanDBDeserializer<ProfileTask> {
+public class ProfileTaskDeserializer extends AbstractBanyanDBDeserializer<ProfileTask> {
     public static final String ID = "profile_task_query_id";
 
-    public ProfileTaskMapper() {
+    public ProfileTaskDeserializer() {
         super(ProfileTaskRecord.INDEX_NAME,
                 ImmutableList.of(ID, ProfileTaskRecord.SERVICE_ID, ProfileTaskRecord.ENDPOINT_NAME,
                         ProfileTaskRecord.START_TIME, ProfileTaskRecord.DURATION, ProfileTaskRecord.MIN_DURATION_THRESHOLD,
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogDeserializer.java
similarity index 62%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogDeserializer.java
index de51d2f95c..79731bf5fb 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileTaskLogDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -9,8 +27,8 @@ import org.apache.skywalking.oap.server.core.query.type.ProfileTaskLogOperationT
 
 import java.util.List;
 
-public class ProfileTaskLogMapper extends AbstractBanyanDBDeserializer<ProfileTaskLog> {
-    public ProfileTaskLogMapper() {
+public class ProfileTaskLogDeserializer extends AbstractBanyanDBDeserializer<ProfileTaskLog> {
+    public ProfileTaskLogDeserializer() {
         super(ProfileTaskLogRecord.INDEX_NAME,
                 ImmutableList.of(ProfileTaskLogRecord.OPERATION_TIME),
                 ImmutableList.of(ProfileTaskLogRecord.TASK_ID, ProfileTaskLogRecord.INSTANCE_ID,
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordDeserializer.java
similarity index 60%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordDeserializer.java
index 2d966a5955..be815f57c0 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ProfileThreadSnapshotRecordDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -9,8 +27,8 @@ import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord
 import java.util.Collections;
 import java.util.List;
 
-public class ProfileThreadSnapshotRecordMapper extends AbstractBanyanDBDeserializer<ProfileThreadSnapshotRecord> {
-    public ProfileThreadSnapshotRecordMapper() {
+public class ProfileThreadSnapshotRecordDeserializer extends AbstractBanyanDBDeserializer<ProfileThreadSnapshotRecord> {
+    public ProfileThreadSnapshotRecordDeserializer() {
         super(ProfileThreadSnapshotRecord.INDEX_NAME,
                 ImmutableList.of(ProfileThreadSnapshotRecord.TASK_ID, ProfileThreadSnapshotRecord.SEGMENT_ID,
                         ProfileThreadSnapshotRecord.DUMP_TIME, ProfileThreadSnapshotRecord.SEQUENCE),
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityDeserializer.java
similarity index 70%
copy from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
copy to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityDeserializer.java
index aef8ec94e7..2e33ed2373 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityDeserializer.java
@@ -16,16 +16,11 @@
  *
  */
 
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
+package org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer;
 
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.skywalking.banyandb.v1.client.StreamWrite;
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.banyandb.v1.client.RowEntity;
 
-@RequiredArgsConstructor
-public class BanyanDBStreamInsertRequest implements InsertRequest {
-
-    @Getter
-    private final StreamWrite streamWrite;
-}
\ No newline at end of file
+@FunctionalInterface
+public interface RowEntityDeserializer<T> {
+    T map(RowEntity row);
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityMapper.java
deleted file mode 100644
index 51f9a5d687..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/RowEntityMapper.java
+++ /dev/null
@@ -1,8 +0,0 @@
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer;
-
-import org.apache.skywalking.banyandb.v1.client.RowEntity;
-
-@FunctionalInterface
-public interface RowEntityMapper<T> {
-    T map(RowEntity row);
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordDeserializer.java
similarity index 62%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordDeserializer.java
index da7be9de35..0665d614af 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/SegmentRecordDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -9,8 +27,8 @@ import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentReco
 import java.util.Collections;
 import java.util.List;
 
-public class SegmentRecordMapper extends AbstractBanyanDBDeserializer<SegmentRecord> {
-    public SegmentRecordMapper() {
+public class SegmentRecordDeserializer extends AbstractBanyanDBDeserializer<SegmentRecord> {
+    public SegmentRecordDeserializer() {
         super(SegmentRecord.INDEX_NAME,
                 ImmutableList.of("trace_id", "state", "service_id", "service_instance_id", "endpoint_id", "duration", "start_time"),
                 Collections.singletonList("data_binary"));
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceDeserializer.java
similarity index 50%
rename from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceMapper.java
rename to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceDeserializer.java
index e006a1107d..cf36fd9239 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceMapper.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceDeserializer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
 
 import com.google.common.collect.ImmutableList;
@@ -8,8 +26,8 @@ import org.apache.skywalking.oap.server.core.query.type.Service;
 
 import java.util.List;
 
-public class ServiceMapper extends AbstractBanyanDBDeserializer<Service> {
-    public ServiceMapper() {
+public class ServiceDeserializer extends AbstractBanyanDBDeserializer<Service> {
+    public ServiceDeserializer() {
         super(ServiceTraffic.INDEX_NAME,
                 ImmutableList.of(ServiceTraffic.NAME, ServiceTraffic.NODE_TYPE, ServiceTraffic.GROUP));
     }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceDeserializer.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceDeserializer.java
new file mode 100644
index 0000000000..a95867c42e
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceDeserializer.java
@@ -0,0 +1,79 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.deserializer;
+
+import com.google.common.collect.ImmutableList;
+import com.google.gson.JsonElement;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.skywalking.banyandb.v1.client.RowEntity;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.instance.InstanceTraffic;
+import org.apache.skywalking.oap.server.core.query.enumeration.Language;
+import org.apache.skywalking.oap.server.core.query.type.Attribute;
+import org.apache.skywalking.oap.server.core.query.type.ServiceInstance;
+import org.apache.skywalking.oap.server.core.remote.grpc.proto.RemoteData;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class ServiceInstanceDeserializer extends AbstractBanyanDBDeserializer<ServiceInstance> {
+    public ServiceInstanceDeserializer() {
+        super(InstanceTraffic.INDEX_NAME,
+                ImmutableList.of(InstanceTraffic.SERVICE_ID, InstanceTraffic.LAST_PING_TIME_BUCKET),
+                Collections.singletonList("data_binary"));
+    }
+
+    @Override
+    public ServiceInstance map(RowEntity row) {
+        InstanceTraffic instanceTraffic = new InstanceTraffic();
+        final List<TagAndValue<?>> data = row.getTagFamilies().get(1);
+        Object o = data.get(0).getValue();
+        ServiceInstance serviceInstance = new ServiceInstance();
+        if (o instanceof ByteString && !((ByteString) o).isEmpty()) {
+            try {
+                RemoteData remoteData = RemoteData.parseFrom((ByteString) o);
+                instanceTraffic.deserialize(remoteData);
+                serviceInstance.setName(instanceTraffic.getName());
+                serviceInstance.setId(instanceTraffic.getServiceId());
+
+                if (instanceTraffic.getProperties() != null) {
+                    for (Map.Entry<String, JsonElement> property : instanceTraffic.getProperties().entrySet()) {
+                        String key = property.getKey();
+                        String value = property.getValue().getAsString();
+                        if (key.equals(InstanceTraffic.PropertyUtil.LANGUAGE)) {
+                            serviceInstance.setLanguage(Language.value(value));
+                        } else {
+                            serviceInstance.getAttributes().add(new Attribute(key, value));
+                        }
+                    }
+                } else {
+                    serviceInstance.setLanguage(Language.UNKNOWN);
+                }
+            } catch (InvalidProtocolBufferException ex) {
+                throw new RuntimeException("fail to parse remote data", ex);
+            }
+        } else {
+            throw new RuntimeException("unable to parse binary data");
+        }
+
+        return serviceInstance;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceMapper.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceMapper.java
deleted file mode 100644
index f6925d3fd3..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/deserializer/ServiceInstanceMapper.java
+++ /dev/null
@@ -1,70 +0,0 @@
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer;
-
-import com.google.common.collect.ImmutableList;
-import com.google.gson.Gson;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.skywalking.banyandb.v1.client.RowEntity;
-import org.apache.skywalking.banyandb.v1.client.TagAndValue;
-import org.apache.skywalking.oap.server.core.analysis.manual.instance.InstanceTraffic;
-import org.apache.skywalking.oap.server.core.query.enumeration.Language;
-import org.apache.skywalking.oap.server.core.query.type.Attribute;
-import org.apache.skywalking.oap.server.core.query.type.ServiceInstance;
-import org.apache.skywalking.oap.server.core.remote.grpc.proto.RemoteData;
-import org.apache.skywalking.oap.server.library.util.StringUtil;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class ServiceInstanceMapper extends AbstractBanyanDBDeserializer<ServiceInstance> {
-    private static final Gson GSON = new Gson();
-
-    public ServiceInstanceMapper() {
-        super(InstanceTraffic.INDEX_NAME,
-                ImmutableList.of(InstanceTraffic.SERVICE_ID, InstanceTraffic.LAST_PING_TIME_BUCKET),
-                Collections.singletonList("data_binary"));
-    }
-
-    @Override
-    public ServiceInstance map(RowEntity row) {
-        ServiceInstance serviceInstance = new ServiceInstance();
-        final List<TagAndValue<?>> searchable = row.getTagFamilies().get(0);
-        serviceInstance.setId((String) searchable.get(0).getValue());
-        serviceInstance.setInstanceUUID((String) searchable.get(0).getValue());
-        final List<TagAndValue<?>> data = row.getTagFamilies().get(1);
-        Object o = data.get(0).getValue();
-        if (o instanceof ByteString && !((ByteString) o).isEmpty()) {
-            try {
-                RemoteData remoteData = RemoteData.parseFrom((ByteString) o);
-                serviceInstance.setName(remoteData.getDataStrings(1));
-                final String propString = remoteData.getDataStrings(2);
-                if (StringUtil.isNotEmpty(propString)) {
-                    JsonObject properties = GSON.fromJson(propString, JsonObject.class);
-                    if (properties != null) {
-                        for (Map.Entry<String, JsonElement> property : properties.entrySet()) {
-                            String key = property.getKey();
-                            String value = property.getValue().getAsString();
-                            if (key.equals(InstanceTraffic.PropertyUtil.LANGUAGE)) {
-                                serviceInstance.setLanguage(Language.value(value));
-                            } else {
-                                serviceInstance.getAttributes().add(new Attribute(key, value));
-                            }
-                        }
-                    } else {
-                        serviceInstance.setLanguage(Language.UNKNOWN);
-                    }
-                } else {
-                    serviceInstance.setLanguage(Language.UNKNOWN);
-                }
-            } catch (InvalidProtocolBufferException ex) {
-                throw new RuntimeException("fail to parse remote data", ex);
-            }
-        } else {
-            serviceInstance.setLanguage(Language.UNKNOWN);
-        }
-        return serviceInstance;
-    }
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/AlarmRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/AlarmRecordBuilder.java
new file mode 100644
index 0000000000..b154c9280d
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/AlarmRecordBuilder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.alarm.AlarmRecord;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class AlarmRecordBuilder extends BanyanDBRecordBuilder<AlarmRecord> {
+    public static final List<String> INDEXED_TAGS = ImmutableList.of(
+            "level"
+    );
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(AlarmRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(2);
+        searchable.add(TagAndValue.longField(entity.getScope()));
+        searchable.add(TagAndValue.longField(entity.getStartTime()));
+        searchable.addAll(filterSearchableTags(entity.getTags(), INDEXED_TAGS));
+        return searchable;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(AlarmRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> data = new ArrayList<>(6);
+        data.add(TagAndValue.stringField(entity.getName()));
+        data.add(TagAndValue.stringField(entity.getId0()));
+        data.add(TagAndValue.stringField(entity.getId1()));
+        data.add(TagAndValue.stringField(entity.getAlarmMessage()));
+        data.add(TagAndValue.stringField(entity.getRuleName()));
+        data.add(TagAndValue.binaryField(entity.getTagsRawData()));
+        return data;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBMetricsBuilder.java
similarity index 64%
copy from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
copy to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBMetricsBuilder.java
index aef8ec94e7..28c3055f47 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBMetricsBuilder.java
@@ -16,16 +16,15 @@
  *
  */
 
-package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
+package org.apache.skywalking.oap.server.storage.plugin.banyandb.schema;
 
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.skywalking.banyandb.v1.client.StreamWrite;
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
 
-@RequiredArgsConstructor
-public class BanyanDBStreamInsertRequest implements InsertRequest {
-
-    @Getter
-    private final StreamWrite streamWrite;
-}
\ No newline at end of file
+public abstract class BanyanDBMetricsBuilder<T extends Metrics> extends BanyanDBStorageDataBuilder<T> {
+    @Override
+    protected long timestamp(Model model, T entity) {
+        return TimeBucket.getTimestamp(entity.getTimeBucket(), model.getDownsampling());
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBRecordBuilder.java
new file mode 100644
index 0000000000..0310c7173b
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBRecordBuilder.java
@@ -0,0 +1,61 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.analysis.manual.searchtag.Tag;
+import org.apache.skywalking.oap.server.core.analysis.record.Record;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BanyanDBRecordBuilder<T extends Record> extends BanyanDBStorageDataBuilder<T> {
+    @Override
+    protected long timestamp(Model model, T entity) {
+        return TimeBucket.getTimestamp(entity.getTimeBucket(), model.getDownsampling());
+    }
+
+    protected List<SerializableTag<Banyandb.TagValue>> filterSearchableTags(List<Tag> rawTags, List<String> indexTags) {
+        if (rawTags == null) {
+            return Collections.emptyList();
+        }
+        Map<String, SerializableTag<Banyandb.TagValue>> map = new HashMap<>();
+        for (final Tag tag : rawTags) {
+            map.put(tag.getKey().toLowerCase(), TagAndValue.stringField(tag.getValue()));
+        }
+        final List<SerializableTag<Banyandb.TagValue>> tags = new ArrayList<>();
+        for (String indexedTag : indexTags) {
+            SerializableTag<Banyandb.TagValue> tag = map.get(indexedTag);
+            if (tag == null) {
+                tags.add(TagAndValue.nullField());
+            } else {
+                tags.add(tag);
+            }
+        }
+
+        return tags;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBStorageDataBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBStorageDataBuilder.java
new file mode 100644
index 0000000000..6d545f8a9e
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BanyanDBStorageDataBuilder.java
@@ -0,0 +1,52 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.StreamWrite;
+import org.apache.skywalking.oap.server.core.storage.StorageData;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+import java.util.Collections;
+import java.util.List;
+
+public abstract class BanyanDBStorageDataBuilder<T extends StorageData> {
+    public StreamWrite entity2Storage(Model model, T entity) {
+        return StreamWrite.builder()
+                .elementId(this.elementID(entity))
+                .name(model.getName())
+                .timestamp(this.timestamp(model, entity))
+                .searchableTags(this.searchableTags(entity))
+                .dataTags(this.dataTags(entity))
+                .build();
+    }
+
+    protected String elementID(T entity) {
+        return entity.id();
+    }
+
+    abstract protected long timestamp(Model model, T entity);
+
+    abstract protected List<SerializableTag<Banyandb.TagValue>> searchableTags(T entity);
+
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(T entity) {
+        return Collections.emptyList();
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BrowserErrorLogRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BrowserErrorLogRecordBuilder.java
new file mode 100644
index 0000000000..0d1923c6f0
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/BrowserErrorLogRecordBuilder.java
@@ -0,0 +1,46 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.browser.manual.errorlog.BrowserErrorLogRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class BrowserErrorLogRecordBuilder extends BanyanDBRecordBuilder<BrowserErrorLogRecord> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(BrowserErrorLogRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>();
+        searchable.add(TagAndValue.stringField(entity.getUniqueId()));
+        searchable.add(TagAndValue.stringField(entity.getServiceId()));
+        searchable.add(TagAndValue.stringField(entity.getServiceVersionId()));
+        searchable.add(TagAndValue.stringField(entity.getPagePathId()));
+        searchable.add(TagAndValue.longField(entity.getErrorCategory()));
+        return searchable;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(BrowserErrorLogRecord entity) {
+        return Collections.singletonList(TagAndValue.binaryField(entity.getDataBinary()));
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/EventBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/EventBuilder.java
new file mode 100644
index 0000000000..cce48e8b71
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/EventBuilder.java
@@ -0,0 +1,41 @@
+package org.apache.skywalking.oap.server.storage.plugin.banyandb.schema;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
+import org.apache.skywalking.oap.server.core.source.Event;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class EventBuilder extends BanyanDBMetricsBuilder<Event> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(Event entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(8);
+        searchable.add(TagAndValue.stringField(entity.getUuid()));
+        searchable.add(TagAndValue.stringField(entity.getService()));
+        searchable.add(TagAndValue.stringField(entity.getServiceInstance()));
+        searchable.add(TagAndValue.stringField(entity.getEndpoint()));
+        searchable.add(TagAndValue.stringField(entity.getName()));
+        searchable.add(TagAndValue.stringField(entity.getType()));
+        searchable.add(TagAndValue.longField(entity.getStartTime()));
+        searchable.add(TagAndValue.longField(entity.getEndTime()));
+        return searchable;
+    }
+
+    @Override
+    protected long timestamp(Model model, Event entity) {
+        return TimeBucket.getTimestamp(entity.getTimeBucket(), model.getDownsampling());
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(Event entity) {
+        return ImmutableList.of(
+                TagAndValue.stringField(entity.getMessage()),
+                TagAndValue.stringField(entity.getParameters())
+        );
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/LogRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/LogRecordBuilder.java
new file mode 100644
index 0000000000..1d88799fd9
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/LogRecordBuilder.java
@@ -0,0 +1,58 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.log.LogRecord;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class LogRecordBuilder extends BanyanDBRecordBuilder<LogRecord> {
+    public static final List<String> INDEXED_TAGS = ImmutableList.of(
+            "level"
+    );
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(LogRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>();
+        searchable.add(TagAndValue.stringField(entity.getUniqueId()));
+        searchable.add(TagAndValue.stringField(entity.getServiceId()));
+        searchable.add(TagAndValue.stringField(entity.getServiceInstanceId()));
+        searchable.add(TagAndValue.stringField(entity.getServiceId()));
+        searchable.add(TagAndValue.stringField(entity.getEndpointId()));
+        searchable.add(TagAndValue.stringField(entity.getTraceId()));
+        searchable.add(TagAndValue.stringField(entity.getTraceSegmentId()));
+        searchable.add(TagAndValue.longField(entity.getSpanId()));
+        searchable.addAll(filterSearchableTags(entity.getTags(), INDEXED_TAGS));
+        return searchable;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(LogRecord entity) {
+        return ImmutableList.of(
+                TagAndValue.stringField(entity.getContent()),
+                TagAndValue.longField(entity.getContentType()),
+                TagAndValue.binaryField(entity.getTagsRawData())
+        );
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/Metadata.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/Metadata.java
new file mode 100644
index 0000000000..1fcd5f1587
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/Metadata.java
@@ -0,0 +1,70 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.endpoint.EndpointTraffic;
+import org.apache.skywalking.oap.server.core.analysis.manual.instance.InstanceTraffic;
+import org.apache.skywalking.oap.server.core.analysis.manual.service.ServiceTraffic;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class Metadata {
+    public static class ServiceTrafficBuilder extends BanyanDBMetricsBuilder<ServiceTraffic> {
+        @Override
+        protected List<SerializableTag<Banyandb.TagValue>> searchableTags(ServiceTraffic entity) {
+            List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(3);
+            searchable.add(TagAndValue.stringField(entity.getName()));
+            searchable.add(TagAndValue.longField(entity.getNodeType().value()));
+            searchable.add(TagAndValue.stringField(entity.getGroup()));
+            return searchable;
+        }
+    }
+
+    public static class EndpointTrafficBuilder extends BanyanDBMetricsBuilder<EndpointTraffic> {
+        @Override
+        protected List<SerializableTag<Banyandb.TagValue>> searchableTags(EndpointTraffic entity) {
+            List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(2);
+            searchable.add(TagAndValue.stringField(entity.getServiceId()));
+            searchable.add(TagAndValue.stringField(entity.getName()));
+            return searchable;
+        }
+    }
+
+    public static class InstanceTrafficBuilder extends BanyanDBMetricsBuilder<InstanceTraffic> {
+        @Override
+        protected List<SerializableTag<Banyandb.TagValue>> searchableTags(InstanceTraffic entity) {
+            List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(2);
+            searchable.add(TagAndValue.stringField(entity.getServiceId()));
+            searchable.add(TagAndValue.longField(entity.getLastPingTimestamp()));
+            return searchable;
+        }
+
+        @Override
+        protected List<SerializableTag<Banyandb.TagValue>> dataTags(InstanceTraffic entity) {
+            return Collections.singletonList(TagAndValue.binaryField(
+                    entity.serialize().build().toByteArray()
+            ));
+        }
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/NetworkAddressAliasBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/NetworkAddressAliasBuilder.java
new file mode 100644
index 0000000000..1054bd922f
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/NetworkAddressAliasBuilder.java
@@ -0,0 +1,44 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.networkalias.NetworkAddressAlias;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class NetworkAddressAliasBuilder extends BanyanDBMetricsBuilder<NetworkAddressAlias> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(NetworkAddressAlias entity) {
+        return Collections.singletonList(TagAndValue.longField(entity.getLastUpdateTimeBucket()));
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(NetworkAddressAlias entity) {
+        List<SerializableTag<Banyandb.TagValue>> data = new ArrayList<>();
+        data.add(TagAndValue.stringField(entity.getAddress()));
+        data.add(TagAndValue.stringField(entity.getRepresentServiceId()));
+        data.add(TagAndValue.stringField(entity.getRepresentServiceInstanceId()));
+        return data;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskLogRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskLogRecordBuilder.java
new file mode 100644
index 0000000000..ee0141dc6d
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskLogRecordBuilder.java
@@ -0,0 +1,44 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskLogRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ProfileTaskLogRecordBuilder extends BanyanDBRecordBuilder<ProfileTaskLogRecord> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(ProfileTaskLogRecord entity) {
+        return Collections.singletonList(TagAndValue.longField(entity.getOperationTime()));
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(ProfileTaskLogRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> data = new ArrayList<>();
+        data.add(TagAndValue.stringField(entity.getTaskId()));
+        data.add(TagAndValue.stringField(entity.getInstanceId()));
+        data.add(TagAndValue.longField(entity.getOperationType()));
+        return data;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskRecordBuilder.java
new file mode 100644
index 0000000000..0ce548ffc6
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileTaskRecordBuilder.java
@@ -0,0 +1,44 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ProfileTaskRecordBuilder extends BanyanDBRecordBuilder<ProfileTaskRecord> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(ProfileTaskRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(9);
+        searchable.add(TagAndValue.stringField(entity.id()));
+        searchable.add(TagAndValue.stringField(entity.getServiceId()));
+        searchable.add(TagAndValue.stringField(entity.getEndpointName()));
+        searchable.add(TagAndValue.longField(entity.getStartTime()));
+        searchable.add(TagAndValue.longField(entity.getDuration()));
+        searchable.add(TagAndValue.longField(entity.getMinDurationThreshold()));
+        searchable.add(TagAndValue.longField(entity.getDuration()));
+        searchable.add(TagAndValue.longField(entity.getCreateTime()));
+        searchable.add(TagAndValue.longField(entity.getMaxSamplingCount()));
+        return searchable;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileThreadSnapshotRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileThreadSnapshotRecordBuilder.java
new file mode 100644
index 0000000000..29f6a8b708
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/ProfileThreadSnapshotRecordBuilder.java
@@ -0,0 +1,45 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class ProfileThreadSnapshotRecordBuilder extends BanyanDBRecordBuilder<ProfileThreadSnapshotRecord> {
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(ProfileThreadSnapshotRecord entity) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(4);
+        searchable.add(TagAndValue.stringField(entity.getTaskId()));
+        searchable.add(TagAndValue.stringField(entity.getSegmentId()));
+        searchable.add(TagAndValue.longField(entity.getDumpTime()));
+        searchable.add(TagAndValue.longField(entity.getSequence()));
+        return searchable;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(ProfileThreadSnapshotRecord entity) {
+        return Collections.singletonList(TagAndValue.binaryField(entity.getStackBinary()));
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/SegmentRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/SegmentRecordBuilder.java
new file mode 100644
index 0000000000..e93335643c
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/SegmentRecordBuilder.java
@@ -0,0 +1,60 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class SegmentRecordBuilder extends BanyanDBRecordBuilder<SegmentRecord> {
+    public static final List<String> INDEXED_TAGS = ImmutableList.of(
+            "http.method",
+            "status_code",
+            "db.type",
+            "db.instance",
+            "mq.queue",
+            "mq.topic",
+            "mq.broker"
+    );
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(SegmentRecord segmentRecord) {
+        List<SerializableTag<Banyandb.TagValue>> searchable = new ArrayList<>(10);
+        searchable.add(TagAndValue.stringField(segmentRecord.getTraceId()));
+        searchable.add(TagAndValue.stringField(segmentRecord.getServiceId()));
+        searchable.add(TagAndValue.stringField(segmentRecord.getServiceInstanceId()));
+        searchable.add(TagAndValue.stringField(segmentRecord.getEndpointId()));
+        searchable.add(TagAndValue.longField(segmentRecord.getStartTime()));
+        searchable.add(TagAndValue.longField(segmentRecord.getLatency()));
+        searchable.add(TagAndValue.longField(segmentRecord.getIsError()));
+        searchable.addAll(filterSearchableTags(segmentRecord.getTagsRawData(), INDEXED_TAGS));
+        return searchable;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(SegmentRecord entity) {
+        return Collections.singletonList(TagAndValue.binaryField(entity.getDataBinary()));
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/UITemplateBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/UITemplateBuilder.java
new file mode 100644
index 0000000000..1776e5af8d
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/schema/UITemplateBuilder.java
@@ -0,0 +1,55 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.schema;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.skywalking.banyandb.v1.Banyandb;
+import org.apache.skywalking.banyandb.v1.client.SerializableTag;
+import org.apache.skywalking.banyandb.v1.client.TagAndValue;
+import org.apache.skywalking.oap.server.core.management.ui.template.UITemplate;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class UITemplateBuilder extends BanyanDBStorageDataBuilder<UITemplate> {
+    public static final long UI_TEMPLATE_TIMESTAMP = 1L;
+
+    @Override
+    protected long timestamp(Model model, UITemplate entity) {
+        return UI_TEMPLATE_TIMESTAMP;
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> searchableTags(UITemplate entity) {
+        return ImmutableList.of(
+                TagAndValue.stringField(entity.getName()),
+                TagAndValue.longField(entity.getDisabled())
+        );
+    }
+
+    @Override
+    protected List<SerializableTag<Banyandb.TagValue>> dataTags(UITemplate entity) {
+        List<SerializableTag<Banyandb.TagValue>> data = new ArrayList<>();
+        data.add(TagAndValue.stringField(entity.getType()));
+        data.add(TagAndValue.stringField(entity.getConfiguration()));
+        data.add(TagAndValue.longField(entity.getActivated()));
+        return data;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/AbstractBanyanDBDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/AbstractBanyanDBDAO.java
index b11cca6ed4..d0daa3375f 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/AbstractBanyanDBDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/AbstractBanyanDBDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.StreamQuery;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBAlarmQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBAlarmQueryDAO.java
index 57f658a380..999d89db79 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBAlarmQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBAlarmQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -8,7 +26,9 @@ import org.apache.skywalking.oap.server.core.analysis.manual.searchtag.Tag;
 import org.apache.skywalking.oap.server.core.query.type.AlarmMessage;
 import org.apache.skywalking.oap.server.core.query.type.Alarms;
 import org.apache.skywalking.oap.server.core.storage.query.IAlarmQueryDAO;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.AlarmRecordBuilder;
 
 import java.io.IOException;
 import java.util.List;
@@ -38,8 +58,13 @@ public class BanyanDBAlarmQueryDAO extends AbstractBanyanDBDAO implements IAlarm
 
                 // TODO: support keyword search
 
-                // TODO: support tag search
-
+                if (CollectionUtils.isNotEmpty(tags)) {
+                    for (final Tag tag : tags) {
+                        if (AlarmRecordBuilder.INDEXED_TAGS.contains(tag.getKey())) {
+                            query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", tag.getKey(), tag.getValue()));
+                        }
+                    }
+                }
                 query.setLimit(limit);
                 query.setOffset(from);
             }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBBrowserLogQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBBrowserLogQueryDAO.java
index 3bd3af8ad5..803b05e255 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBBrowserLogQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBBrowserLogQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -25,16 +43,12 @@ public class BanyanDBBrowserLogQueryDAO extends AbstractBanyanDBDAO implements I
 
     @Override
     public BrowserErrorLogs queryBrowserErrorLogs(String serviceId, String serviceVersionId, String pagePathId, BrowserErrorCategory category, long startSecondTB, long endSecondTB, int limit, int from) throws IOException {
-        final BrowserErrorLogs logs = new BrowserErrorLogs();
-        List<BrowserErrorLog> browserErrorLogs = query(BrowserErrorLog.class, new QueryBuilder() {
+
+        final QueryBuilder qb = new QueryBuilder() {
             @Override
             public void apply(StreamQuery query) {
                 query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", BrowserErrorLogRecord.SERVICE_ID, serviceId));
 
-                if (startSecondTB != 0 && endSecondTB != 0) {
-                    query.appendCondition(PairQueryCondition.LongQueryCondition.ge("searchable", BrowserErrorLogRecord.TIMESTAMP, TimeBucket.getTimestamp(startSecondTB)));
-                    query.appendCondition(PairQueryCondition.LongQueryCondition.le("searchable", BrowserErrorLogRecord.TIMESTAMP, TimeBucket.getTimestamp(endSecondTB)));
-                }
                 if (StringUtil.isNotEmpty(serviceVersionId)) {
                     query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", BrowserErrorLogRecord.SERVICE_VERSION_ID, serviceVersionId));
                 }
@@ -48,7 +62,15 @@ public class BanyanDBBrowserLogQueryDAO extends AbstractBanyanDBDAO implements I
                 query.setOffset(from);
                 query.setLimit(limit);
             }
-        });
+        };
+
+        final BrowserErrorLogs logs = new BrowserErrorLogs();
+        final List<BrowserErrorLog> browserErrorLogs;
+        if (startSecondTB != 0 && endSecondTB != 0) {
+            browserErrorLogs = query(BrowserErrorLog.class, qb, TimeBucket.getTimestamp(startSecondTB), TimeBucket.getTimestamp(endSecondTB));
+        } else {
+            browserErrorLogs = query(BrowserErrorLog.class, qb);
+        }
         logs.getLogs().addAll(browserErrorLogs);
         logs.setTotal(logs.getLogs().size());
         return logs;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBEventQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBEventQueryDAO.java
index 9b2f0d2be5..b85e07d860 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBEventQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBEventQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import com.google.common.base.Strings;
@@ -45,7 +63,6 @@ public class BanyanDBEventQueryDAO extends AbstractBanyanDBDAO implements IEvent
 
         Events events = new Events();
         events.setEvents(eventList);
-        // TODO: how to set total???
         events.setTotal(eventList.size());
         return events;
     }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBLogQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBLogQueryDAO.java
index 404d60017c..8325618f88 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBLogQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBLogQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -10,8 +28,10 @@ import org.apache.skywalking.oap.server.core.query.input.TraceScopeCondition;
 import org.apache.skywalking.oap.server.core.query.type.Log;
 import org.apache.skywalking.oap.server.core.query.type.Logs;
 import org.apache.skywalking.oap.server.core.storage.query.ILogQueryDAO;
+import org.apache.skywalking.oap.server.library.util.CollectionUtils;
 import org.apache.skywalking.oap.server.library.util.StringUtil;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.LogRecordBuilder;
 
 import java.io.IOException;
 import java.util.List;
@@ -30,18 +50,13 @@ public class BanyanDBLogQueryDAO extends AbstractBanyanDBDAO implements ILogQuer
                           TraceScopeCondition relatedTrace, Order queryOrder, int from, int limit,
                           long startTB, long endTB, List<Tag> tags, List<String> keywordsOfContent,
                           List<String> excludingKeywordsOfContent) throws IOException {
-        List<Log> entities = query(Log.class, new QueryBuilder() {
+        final QueryBuilder query = new QueryBuilder() {
             @Override
             public void apply(StreamQuery query) {
                 if (StringUtil.isNotEmpty(serviceId)) {
                     query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", AbstractLogRecord.SERVICE_ID, serviceId));
                 }
 
-                if (startTB != 0 && endTB != 0) {
-                    query.appendCondition(PairQueryCondition.LongQueryCondition.ge("searchable", AbstractLogRecord.TIMESTAMP, TimeBucket.getTimestamp(startTB)));
-                    query.appendCondition(PairQueryCondition.LongQueryCondition.le("searchable", AbstractLogRecord.TIMESTAMP, TimeBucket.getTimestamp(endTB)));
-                }
-
                 if (StringUtil.isNotEmpty(serviceInstanceId)) {
                     query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", AbstractLogRecord.SERVICE_INSTANCE_ID, serviceInstanceId));
                 }
@@ -60,14 +75,22 @@ public class BanyanDBLogQueryDAO extends AbstractBanyanDBDAO implements ILogQuer
                     }
                 }
 
-                // TODO: if we allow to index tags?
-//        if (CollectionUtils.isNotEmpty(tags)) {
-//            for (final Tag tag : tags) {
-//                query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", tag.getKey(), tag.getValue()));
-//            }
-//        }
+                if (CollectionUtils.isNotEmpty(tags)) {
+                    for (final Tag tag : tags) {
+                        if (LogRecordBuilder.INDEXED_TAGS.contains(tag.getKey())) {
+                            query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", tag.getKey(), tag.getValue()));
+                        }
+                    }
+                }
             }
-        });
+        };
+
+        final List<Log> entities;
+        if (startTB != 0 && endTB != 0) {
+            entities = query(Log.class, query, TimeBucket.getTimestamp(startTB), TimeBucket.getTimestamp(endTB));
+        } else {
+            entities = query(Log.class, query);
+        }
         Logs logs = new Logs();
         logs.getLogs().addAll(entities);
         logs.setTotal(entities.size());
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBManagementDAO.java
similarity index 53%
copy from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
copy to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBManagementDAO.java
index aef8ec94e7..aecdd13dfd 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBManagementDAO.java
@@ -18,14 +18,24 @@
 
 package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
 
-import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.skywalking.banyandb.v1.client.StreamWrite;
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.core.analysis.management.ManagementData;
+import org.apache.skywalking.oap.server.core.storage.IManagementDAO;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.BanyanDBStorageDataBuilder;
+
+import java.io.IOException;
 
 @RequiredArgsConstructor
-public class BanyanDBStreamInsertRequest implements InsertRequest {
+public class BanyanDBManagementDAO<T extends ManagementData> implements IManagementDAO {
+    private final BanyanDBStorageClient client;
+    private final BanyanDBStorageDataBuilder<T> storageBuilder;
 
-    @Getter
-    private final StreamWrite streamWrite;
-}
\ No newline at end of file
+    @Override
+    public void insert(Model model, ManagementData storageData) throws IOException {
+        StreamWrite streamWrite = this.storageBuilder.entity2Storage(model, (T) storageData);
+        client.write(streamWrite);
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetadataQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetadataQueryDAO.java
index dd3d9623a6..bee6508cd2 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetadataQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetadataQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetricsDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetricsDAO.java
new file mode 100644
index 0000000000..618ff86af9
--- /dev/null
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBMetricsDAO.java
@@ -0,0 +1,53 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.skywalking.banyandb.v1.client.StreamWrite;
+import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.BanyanDBMetricsBuilder;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+@RequiredArgsConstructor
+public class BanyanDBMetricsDAO<T extends Metrics> implements IMetricsDAO {
+    private final BanyanDBMetricsBuilder<T> storageBuilder;
+
+    @Override
+    public List<Metrics> multiGet(Model model, List<Metrics> metrics) throws IOException {
+        return Collections.emptyList();
+    }
+
+    @Override
+    public InsertRequest prepareBatchInsert(Model model, Metrics metrics) throws IOException {
+        StreamWrite streamWrite = this.storageBuilder.entity2Storage(model, (T) metrics);
+        return new BanyanDBStreamInsertRequest(streamWrite);
+    }
+
+    @Override
+    public UpdateRequest prepareBatchUpdate(Model model, Metrics metrics) throws IOException {
+        return null;
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNetworkAddressAliasDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNetworkAddressAliasDAO.java
index 6112593a55..ce36573617 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNetworkAddressAliasDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNetworkAddressAliasDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNoneStreamDAO.java
similarity index 54%
copy from oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
copy to oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNoneStreamDAO.java
index aef8ec94e7..30ed279432 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBNoneStreamDAO.java
@@ -18,14 +18,24 @@
 
 package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
 
-import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.skywalking.banyandb.v1.client.StreamWrite;
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
+import org.apache.skywalking.oap.server.core.analysis.config.NoneStream;
+import org.apache.skywalking.oap.server.core.storage.INoneStreamDAO;
+import org.apache.skywalking.oap.server.core.storage.model.Model;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.BanyanDBRecordBuilder;
+
+import java.io.IOException;
 
 @RequiredArgsConstructor
-public class BanyanDBStreamInsertRequest implements InsertRequest {
+public class BanyanDBNoneStreamDAO<T extends NoneStream> implements INoneStreamDAO {
+    private final BanyanDBStorageClient client;
+    private final BanyanDBRecordBuilder<T> storageBuilder;
 
-    @Getter
-    private final StreamWrite streamWrite;
-}
\ No newline at end of file
+    @Override
+    public void insert(Model model, NoneStream noneStream) throws IOException {
+        StreamWrite streamWrite = this.storageBuilder.entity2Storage(model, (T) noneStream);
+        this.client.write(streamWrite);
+    }
+}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskLogQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskLogQueryDAO.java
index 0095965109..3a3c12e119 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskLogQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskLogQueryDAO.java
@@ -1,11 +1,27 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.StreamQuery;
 import org.apache.skywalking.oap.server.core.query.type.ProfileTaskLog;
 import org.apache.skywalking.oap.server.core.storage.profile.IProfileTaskLogQueryDAO;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.ProfileTaskLogMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.RowEntityMapper;
 
 import java.io.IOException;
 import java.util.Comparator;
@@ -16,8 +32,6 @@ import java.util.stream.Collectors;
  * {@link org.apache.skywalking.oap.server.core.profile.ProfileTaskLogRecord} is a stream
  */
 public class BanyanDBProfileTaskLogQueryDAO extends AbstractBanyanDBDAO implements IProfileTaskLogQueryDAO {
-    private static final RowEntityMapper<ProfileTaskLog> MAPPER = new ProfileTaskLogMapper();
-
     private final int queryMaxSize;
 
     public BanyanDBProfileTaskLogQueryDAO(BanyanDBStorageClient client, int queryMaxSize) {
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskQueryDAO.java
index 23fb8c7d1b..47c7c57a00 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileTaskQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -8,8 +26,7 @@ import org.apache.skywalking.oap.server.core.query.type.ProfileTask;
 import org.apache.skywalking.oap.server.core.storage.profile.IProfileTaskQueryDAO;
 import org.apache.skywalking.oap.server.library.util.StringUtil;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.ProfileTaskMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.RowEntityMapper;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.ProfileTaskDeserializer;
 
 import java.io.IOException;
 import java.util.List;
@@ -19,8 +36,6 @@ import java.util.Objects;
  * {@link org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord} is a stream
  */
 public class BanyanDBProfileTaskQueryDAO extends AbstractBanyanDBDAO implements IProfileTaskQueryDAO {
-    private static final RowEntityMapper<ProfileTask> MAPPER = new ProfileTaskMapper();
-
     public BanyanDBProfileTaskQueryDAO(BanyanDBStorageClient client) {
         super(client);
     }
@@ -68,7 +83,7 @@ public class BanyanDBProfileTaskQueryDAO extends AbstractBanyanDBDAO implements
         return query(ProfileTask.class, new QueryBuilder() {
             @Override
             public void apply(StreamQuery query) {
-                query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", ProfileTaskMapper.ID, id));
+                query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", ProfileTaskDeserializer.ID, id));
                 query.setLimit(1);
             }
         }).stream().findAny().orElse(null);
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileThreadSnapshotQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileThreadSnapshotQueryDAO.java
index e92a0b24ff..c6b9cdec9f 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileThreadSnapshotQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBProfileThreadSnapshotQueryDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -7,10 +25,6 @@ import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord
 import org.apache.skywalking.oap.server.core.query.type.BasicTrace;
 import org.apache.skywalking.oap.server.core.storage.profile.IProfileThreadSnapshotQueryDAO;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.BasicTraceMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.ProfileThreadSnapshotRecordMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.RowEntityMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.SegmentRecordMapper;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -24,10 +38,6 @@ import java.util.stream.Collectors;
  * {@link ProfileThreadSnapshotRecord} is a stream
  */
 public class BanyanDBProfileThreadSnapshotQueryDAO extends AbstractBanyanDBDAO implements IProfileThreadSnapshotQueryDAO {
-    private static final RowEntityMapper<ProfileThreadSnapshotRecord> MAPPER = new ProfileThreadSnapshotRecordMapper();
-    private static final RowEntityMapper<BasicTrace> BASIC_TRACE_MAPPER = new BasicTraceMapper();
-    private static final RowEntityMapper<SegmentRecord> SEGMENT_RECORD_MAPPER = new SegmentRecordMapper();
-
     public BanyanDBProfileThreadSnapshotQueryDAO(BanyanDBStorageClient client) {
         super(client);
     }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBRecordDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBRecordDAO.java
index 54870d98c6..0bd0e30a82 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBRecordDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBRecordDAO.java
@@ -19,61 +19,23 @@
 package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.skywalking.banyandb.v1.Banyandb;
-import org.apache.skywalking.banyandb.v1.client.SerializableTag;
 import org.apache.skywalking.banyandb.v1.client.StreamWrite;
-import org.apache.skywalking.banyandb.v1.client.Tag;
-import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
 import org.apache.skywalking.oap.server.core.analysis.record.Record;
 import org.apache.skywalking.oap.server.core.storage.IRecordDAO;
-import org.apache.skywalking.oap.server.core.storage.StorageHashMapBuilder;
 import org.apache.skywalking.oap.server.core.storage.model.Model;
 import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBSchema;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.BanyanDBRecordBuilder;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
 
 @RequiredArgsConstructor
-public class BanyanDBRecordDAO implements IRecordDAO {
-    private final StorageHashMapBuilder<Record> storageBuilder;
+public class BanyanDBRecordDAO<T extends Record> implements IRecordDAO {
+    private final BanyanDBRecordBuilder<T> storageBuilder;
 
     @Override
     public InsertRequest prepareBatchInsert(Model model, Record record) throws IOException {
-        if (SegmentRecord.INDEX_NAME.equals(model.getName())) {
-            SegmentRecord segmentRecord = (SegmentRecord) record;
-            StreamWrite streamWrite = StreamWrite.builder()
-                    .name(SegmentRecord.INDEX_NAME)
-                    .dataTag(Tag.binaryField(segmentRecord.getDataBinary()))
-                    .timestamp(segmentRecord.getStartTime())
-                    .elementId(segmentRecord.getSegmentId())
-                    .searchableTags(buildFieldObjects(this.storageBuilder.entity2Storage(segmentRecord)))
-                    .build();
-            return new BanyanDBStreamInsertRequest(streamWrite);
-        }
-        // TODO: support other stream types
-        return new InsertRequest() {
-        };
-    }
+        StreamWrite streamWrite = storageBuilder.entity2Storage(model, (T) record);
 
-    /**
-     * Convert storageEntity in Map to an ordered list of {@link SerializableTag}
-     *
-     * @param segmentRecordMap which comes from {@link SegmentRecord}
-     * @return an ordered list of {@link SerializableTag} which is accepted by BanyanDB Client
-     */
-    static List<SerializableTag<Banyandb.TagValue>> buildFieldObjects(Map<String, Object> segmentRecordMap) {
-        List<SerializableTag<Banyandb.TagValue>> tagList = new ArrayList<>(BanyanDBSchema.FIELD_NAMES.size());
-        for (String fieldName : BanyanDBSchema.FIELD_NAMES) {
-            Object val = segmentRecordMap.get(fieldName);
-            if (val == null) {
-                tagList.add(Tag.nullField());
-            } else {
-                tagList.add((SerializableTag<Banyandb.TagValue>) val);
-            }
-        }
-        return tagList;
+        return new BanyanDBStreamInsertRequest(streamWrite);
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBSegmentRecordBuilder.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBSegmentRecordBuilder.java
deleted file mode 100644
index b8c4744380..0000000000
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBSegmentRecordBuilder.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.oap.server.storage.plugin.banyandb.stream;
-
-import org.apache.skywalking.banyandb.v1.client.TagAndValue;
-import org.apache.skywalking.oap.server.core.analysis.manual.searchtag.Tag;
-import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
-import org.apache.skywalking.oap.server.core.analysis.record.Record;
-import org.apache.skywalking.oap.server.core.storage.StorageHashMapBuilder;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class BanyanDBSegmentRecordBuilder implements StorageHashMapBuilder<Record> {
-    @Override
-    public SegmentRecord storage2Entity(Map<String, Object> dbMap) {
-        return new SegmentRecord();
-    }
-
-    /**
-     * Map SegmentRecord to Skywalking-BanyanDB compatible Map with indexed tags and
-     * without binaryData, entityId
-     */
-    @Override
-    public Map<String, Object> entity2Storage(Record record) {
-        final SegmentRecord segmentRecord = (SegmentRecord) record;
-        Map<String, Object> map = new HashMap<>();
-        map.put(SegmentRecord.TRACE_ID, TagAndValue.stringField(segmentRecord.getTraceId()));
-        map.put(SegmentRecord.SERVICE_ID, TagAndValue.stringField(segmentRecord.getServiceId()));
-        map.put(SegmentRecord.SERVICE_INSTANCE_ID, TagAndValue.stringField(segmentRecord.getServiceInstanceId()));
-        map.put(SegmentRecord.ENDPOINT_ID, TagAndValue.stringField(segmentRecord.getEndpointId()));
-        map.put(SegmentRecord.START_TIME, TagAndValue.longField(segmentRecord.getStartTime()));
-        map.put("duration", TagAndValue.longField(segmentRecord.getLatency()));
-        map.put("state", TagAndValue.longField(segmentRecord.getIsError()));
-        if (segmentRecord.getTagsRawData() != null) {
-            for (final Tag tag : segmentRecord.getTagsRawData()) {
-                map.put(tag.getKey().toLowerCase(), TagAndValue.stringField(tag.getValue()));
-            }
-        }
-        return map;
-    }
-}
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStorageDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStorageDAO.java
index be81608a37..76c0359153 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStorageDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStorageDAO.java
@@ -19,55 +19,88 @@
 package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
 
 import lombok.extern.slf4j.Slf4j;
+import org.apache.skywalking.oap.server.core.alarm.AlarmRecord;
+import org.apache.skywalking.oap.server.core.analysis.manual.endpoint.EndpointTraffic;
+import org.apache.skywalking.oap.server.core.analysis.manual.instance.InstanceTraffic;
+import org.apache.skywalking.oap.server.core.analysis.manual.log.LogRecord;
+import org.apache.skywalking.oap.server.core.analysis.manual.networkalias.NetworkAddressAlias;
 import org.apache.skywalking.oap.server.core.analysis.manual.segment.SegmentRecord;
-import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics;
+import org.apache.skywalking.oap.server.core.analysis.manual.service.ServiceTraffic;
+import org.apache.skywalking.oap.server.core.browser.manual.errorlog.BrowserErrorLogRecord;
+import org.apache.skywalking.oap.server.core.management.ui.template.UITemplate;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskLogRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileTaskRecord;
+import org.apache.skywalking.oap.server.core.profile.ProfileThreadSnapshotRecord;
+import org.apache.skywalking.oap.server.core.source.Event;
+import org.apache.skywalking.oap.server.core.storage.AbstractDAO;
 import org.apache.skywalking.oap.server.core.storage.IManagementDAO;
 import org.apache.skywalking.oap.server.core.storage.IMetricsDAO;
 import org.apache.skywalking.oap.server.core.storage.INoneStreamDAO;
 import org.apache.skywalking.oap.server.core.storage.IRecordDAO;
 import org.apache.skywalking.oap.server.core.storage.StorageDAO;
-import org.apache.skywalking.oap.server.core.storage.model.Model;
 import org.apache.skywalking.oap.server.core.storage.type.StorageBuilder;
-import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
-import org.apache.skywalking.oap.server.library.client.request.UpdateRequest;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.AlarmRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.BrowserErrorLogRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.EventBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.LogRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.Metadata;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.NetworkAddressAliasBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.ProfileTaskLogRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.ProfileTaskRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.ProfileThreadSnapshotRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.SegmentRecordBuilder;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.UITemplateBuilder;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 
 @Slf4j
-public class BanyanDBStorageDAO implements StorageDAO {
+public class BanyanDBStorageDAO extends AbstractDAO<BanyanDBStorageClient> implements StorageDAO {
+    public BanyanDBStorageDAO(BanyanDBStorageClient client) {
+        super(client);
+    }
+
     @Override
     public IMetricsDAO newMetricsDao(StorageBuilder storageBuilder) {
-        return new IMetricsDAO() {
-            @Override
-            public List<Metrics> multiGet(Model model, List<Metrics> metrics) throws IOException {
-                return Collections.emptyList();
-            }
-
-            @Override
-            public InsertRequest prepareBatchInsert(Model model, Metrics metrics) throws IOException {
-                return new InsertRequest() {
-                };
-            }
-
-            @Override
-            public UpdateRequest prepareBatchUpdate(Model model, Metrics metrics) throws IOException {
-                return new UpdateRequest() {
-                };
+        try {
+            Class<?> returnType = storageBuilder.getClass().getMethod("storage2Entity", Map.class).getReturnType();
+            if (Event.class.equals(returnType)) {
+                return new BanyanDBMetricsDAO<>(new EventBuilder());
+            } else if (ServiceTraffic.class.equals(returnType)) {
+                return new BanyanDBMetricsDAO<>(new Metadata.ServiceTrafficBuilder());
+            } else if (InstanceTraffic.class.equals(returnType)) {
+                return new BanyanDBMetricsDAO<>(new Metadata.InstanceTrafficBuilder());
+            } else if (EndpointTraffic.class.equals(returnType)) {
+                return new BanyanDBMetricsDAO<>(new Metadata.EndpointTrafficBuilder());
+            } else if (NetworkAddressAlias.class.equals(returnType)) {
+                return new BanyanDBMetricsDAO<>(new NetworkAddressAliasBuilder());
+            } else {
+                throw new IllegalStateException("record type is not supported");
             }
-        };
+        } catch (NoSuchMethodException noSuchMethodException) {
+            log.error("cannot find method storage2Entity", noSuchMethodException);
+            throw new RuntimeException("cannot find method storage2Entity");
+        }
     }
 
     @Override
     public IRecordDAO newRecordDao(StorageBuilder storageBuilder) {
         try {
-            if (SegmentRecord.class.equals(storageBuilder.getClass().getMethod("storage2Entity", Map.class).getReturnType())) {
-                return new BanyanDBRecordDAO(new BanyanDBSegmentRecordBuilder());
+            Class<?> returnType = storageBuilder.getClass().getMethod("storage2Entity", Map.class).getReturnType();
+            if (SegmentRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new SegmentRecordBuilder());
+            } else if (AlarmRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new AlarmRecordBuilder());
+            } else if (BrowserErrorLogRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new BrowserErrorLogRecordBuilder());
+            } else if (LogRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new LogRecordBuilder());
+            } else if (ProfileTaskLogRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new ProfileTaskLogRecordBuilder());
+            } else if (ProfileThreadSnapshotRecord.class.equals(returnType)) {
+                return new BanyanDBRecordDAO<>(new ProfileThreadSnapshotRecordBuilder());
             } else {
-                return (model, record) -> new InsertRequest() {
-                };
+                throw new IllegalStateException("record type is not supported");
             }
         } catch (NoSuchMethodException noSuchMethodException) {
             log.error("cannot find method storage2Entity", noSuchMethodException);
@@ -77,13 +110,31 @@ public class BanyanDBStorageDAO implements StorageDAO {
 
     @Override
     public INoneStreamDAO newNoneStreamDao(StorageBuilder storageBuilder) {
-        return (model, noneStream) -> {
-        };
+        try {
+            Class<?> returnType = storageBuilder.getClass().getMethod("storage2Entity", Map.class).getReturnType();
+            if (ProfileTaskRecord.class.equals(returnType)) {
+                return new BanyanDBNoneStreamDAO<>(getClient(), new ProfileTaskRecordBuilder());
+            } else {
+                throw new IllegalStateException("record type is not supported");
+            }
+        } catch (NoSuchMethodException noSuchMethodException) {
+            log.error("cannot find method storage2Entity", noSuchMethodException);
+            throw new RuntimeException("cannot find method storage2Entity");
+        }
     }
 
     @Override
     public IManagementDAO newManagementDao(StorageBuilder storageBuilder) {
-        return (model, storageData) -> {
-        };
+        try {
+            Class<?> returnType = storageBuilder.getClass().getMethod("storage2Entity", Map.class).getReturnType();
+            if (UITemplate.class.equals(returnType)) {
+                return new BanyanDBManagementDAO<>(getClient(), new UITemplateBuilder());
+            } else {
+                throw new IllegalStateException("record type is not supported");
+            }
+        } catch (NoSuchMethodException noSuchMethodException) {
+            log.error("cannot find method storage2Entity", noSuchMethodException);
+            throw new RuntimeException("cannot find method storage2Entity");
+        }
     }
 }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
index aef8ec94e7..f8287973bc 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBStreamInsertRequest.java
@@ -25,7 +25,6 @@ import org.apache.skywalking.oap.server.library.client.request.InsertRequest;
 
 @RequiredArgsConstructor
 public class BanyanDBStreamInsertRequest implements InsertRequest {
-
     @Getter
     private final StreamWrite streamWrite;
 }
\ No newline at end of file
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBTraceQueryDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBTraceQueryDAO.java
index bbfd4e3150..73e80c8f94 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBTraceQueryDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBTraceQueryDAO.java
@@ -19,6 +19,7 @@
 package org.apache.skywalking.oap.server.storage.plugin.banyandb.stream;
 
 import com.google.common.base.Strings;
+import lombok.Getter;
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
 import org.apache.skywalking.banyandb.v1.client.StreamQuery;
 import org.apache.skywalking.oap.server.core.analysis.TimeBucket;
@@ -31,24 +32,14 @@ import org.apache.skywalking.oap.server.core.query.type.TraceBrief;
 import org.apache.skywalking.oap.server.core.query.type.TraceState;
 import org.apache.skywalking.oap.server.core.storage.query.ITraceQueryDAO;
 import org.apache.skywalking.oap.server.library.util.CollectionUtils;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBSchema;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.BasicTraceMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.RowEntityMapper;
-import org.apache.skywalking.oap.server.storage.plugin.banyandb.deserializer.SegmentRecordMapper;
-import org.joda.time.format.DateTimeFormat;
-import org.joda.time.format.DateTimeFormatter;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.SegmentRecordBuilder;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
 public class BanyanDBTraceQueryDAO extends AbstractBanyanDBDAO implements ITraceQueryDAO {
-    private static final RowEntityMapper<SegmentRecord> SEGMENT_RECORD_MAPPER = new SegmentRecordMapper();
-    private static final RowEntityMapper<BasicTrace> BASIC_TRACE_MAPPER = new BasicTraceMapper();
-
-    private static final DateTimeFormatter YYYYMMDDHHMMSS = DateTimeFormat.forPattern("yyyyMMddHHmmss");
-
     public BanyanDBTraceQueryDAO(BanyanDBStorageClient client) {
         super(client);
     }
@@ -81,13 +72,13 @@ public class BanyanDBTraceQueryDAO extends AbstractBanyanDBDAO implements ITrace
 
                 switch (traceState) {
                     case ERROR:
-                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) BanyanDBSchema.TraceState.ERROR.getState()));
+                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) TraceStateStorage.ERROR.getState()));
                         break;
                     case SUCCESS:
-                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) BanyanDBSchema.TraceState.SUCCESS.getState()));
+                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) TraceStateStorage.SUCCESS.getState()));
                         break;
                     default:
-                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) BanyanDBSchema.TraceState.ALL.getState()));
+                        query.appendCondition(PairQueryCondition.LongQueryCondition.eq("searchable", "state", (long) TraceStateStorage.ALL.getState()));
                         break;
                 }
 
@@ -102,7 +93,7 @@ public class BanyanDBTraceQueryDAO extends AbstractBanyanDBDAO implements ITrace
 
                 if (CollectionUtils.isNotEmpty(tags)) {
                     for (final Tag tag : tags) {
-                        if (BanyanDBSchema.INDEX_FIELDS.contains(tag.getKey())) {
+                        if (SegmentRecordBuilder.INDEXED_TAGS.contains(tag.getKey())) {
                             query.appendCondition(PairQueryCondition.StringQueryCondition.eq("searchable", tag.getKey(), tag.getValue()));
                         }
                     }
@@ -140,4 +131,15 @@ public class BanyanDBTraceQueryDAO extends AbstractBanyanDBDAO implements ITrace
     public List<Span> doFlexibleTraceQuery(String traceId) throws IOException {
         return Collections.emptyList();
     }
+
+    public enum TraceStateStorage {
+        ALL(0), SUCCESS(1), ERROR(2);
+
+        @Getter
+        private final int state;
+
+        TraceStateStorage(int state) {
+            this.state = state;
+        }
+    }
 }
diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBUITemplateManagementDAO.java b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBUITemplateManagementDAO.java
index 95731220fd..9ed0af8447 100644
--- a/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBUITemplateManagementDAO.java
+++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/banyandb/stream/BanyanDBUITemplateManagementDAO.java
@@ -1,3 +1,21 @@
+/*
+ * 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.oap.server.storage.plugin.banyandb.stream;
 
 import org.apache.skywalking.banyandb.v1.client.PairQueryCondition;
@@ -11,6 +29,7 @@ import org.apache.skywalking.oap.server.core.query.type.TemplateChangeStatus;
 import org.apache.skywalking.oap.server.core.storage.management.UITemplateManagementDAO;
 import org.apache.skywalking.oap.server.library.util.BooleanUtils;
 import org.apache.skywalking.oap.server.storage.plugin.banyandb.BanyanDBStorageClient;
+import org.apache.skywalking.oap.server.storage.plugin.banyandb.schema.UITemplateBuilder;
 
 import java.io.IOException;
 import java.util.List;
@@ -19,8 +38,6 @@ import java.util.List;
  * {@link org.apache.skywalking.oap.server.core.management.ui.template.UITemplate} is a stream
  */
 public class BanyanDBUITemplateManagementDAO extends AbstractBanyanDBDAO implements UITemplateManagementDAO {
-    private static final long UI_TEMPLATE_TIMESTAMP = 1L;
-
     public BanyanDBUITemplateManagementDAO(BanyanDBStorageClient client) {
         super(client);
     }
@@ -54,7 +71,7 @@ public class BanyanDBUITemplateManagementDAO extends AbstractBanyanDBDAO impleme
                 .dataTag(Tag.stringField(uiTemplate.getConfiguration()))
                 // data - activated
                 .dataTag(Tag.longField(uiTemplate.getActivated()))
-                .timestamp(UI_TEMPLATE_TIMESTAMP)
+                .timestamp(UITemplateBuilder.UI_TEMPLATE_TIMESTAMP)
                 .elementId(uiTemplate.id())
                 .build();
         getClient().write(request);